merged back master to jarodwen/hotfix/issue461
diff --git a/asterix-algebra/pom.xml b/asterix-algebra/pom.xml
index 350734b..86d621b 100644
--- a/asterix-algebra/pom.xml
+++ b/asterix-algebra/pom.xml
@@ -92,6 +92,12 @@
 			<version>0.0.6-SNAPSHOT</version>
 			<scope>compile</scope>
 		</dependency>
+                <dependency>
+                        <groupId>edu.uci.ics.asterix</groupId>
+                        <artifactId>asterix-transactions</artifactId>
+                        <version>0.0.6-SNAPSHOT</version>
+                        <scope>compile</scope>
+                </dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>algebricks-compiler</artifactId>
@@ -99,9 +105,6 @@
 		<dependency>
 			<groupId>org.json</groupId>
 			<artifactId>json</artifactId>
-			<version>20090211</version>
-			<type>jar</type>
-			<scope>compile</scope>
 		</dependency>
 	</dependencies>
 </project>
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitPOperator.java
index ece3db6..5f3bb36 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitPOperator.java
@@ -17,7 +17,7 @@
 
 import java.util.List;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
index 8533e74..266fe6d 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
@@ -17,13 +17,13 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext.TransactionType;
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -46,7 +46,7 @@
     private final boolean isWriteTransaction;
     private final long[] longHashes; 
 
-    private TransactionContext transactionContext;
+    private ITransactionContext transactionContext;
     private RecordDescriptor inputRecordDesc;
     private FrameTupleAccessor frameTupleAccessor;
     private FrameTupleReference frameTupleReference;
@@ -54,7 +54,7 @@
     public CommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
             boolean isWriteTransaction) {
         this.hyracksTaskCtx = ctx;
-        AsterixAppRuntimeContext runtimeCtx = (AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+        IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
                 .getApplicationObject();
         this.transactionManager = runtimeCtx.getTransactionSubsystem().getTransactionManager();
         this.jobId = jobId;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntimeFactory.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntimeFactory.java
index f3b6526..39ba697 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntimeFactory.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntimeFactory.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.asterix.algebra.operators.physical;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 6edcc39..da2e838 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -3,9 +3,8 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
-import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -20,10 +19,12 @@
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
 import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod.SearchModifierType;
 import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexJobGenParams;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
@@ -201,19 +202,19 @@
             AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
             if (!isPartitioned) {
                 dataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(
+                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                         storageProperties.getBloomFilterFalsePositiveRate());
             } else {
                 dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
+                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                         storageProperties.getBloomFilterFalsePositiveRate());
             }
             LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index 783fbaa..e13da67 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -34,7 +34,8 @@
 import edu.uci.ics.asterix.optimizer.rules.IntroduceInstantLockSearchCallbackRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceRapidFrameFlushProjectRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
-import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastForInsertRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceUnnestForCollectionToSequenceRule;
 import edu.uci.ics.asterix.optimizer.rules.LoadRecordFieldsRule;
 import edu.uci.ics.asterix.optimizer.rules.NestGroupByRule;
 import edu.uci.ics.asterix.optimizer.rules.NestedSubplanToJoinRule;
@@ -110,6 +111,7 @@
 
     public final static List<IAlgebraicRewriteRule> buildNormalizationRuleCollection() {
         List<IAlgebraicRewriteRule> normalization = new LinkedList<IAlgebraicRewriteRule>();
+        normalization.add(new IntroduceUnnestForCollectionToSequenceRule());
         normalization.add(new EliminateSubplanRule());
         normalization.add(new EnforceOrderByAfterSubplan());
         normalization.add(new PushAggFuncIntoStandaloneAggregateRule());
@@ -122,7 +124,7 @@
         // IntroduceStaticTypeCastRule should go before
         // IntroduceDynamicTypeCastRule to
         // avoid unnecessary dynamic casting
-        normalization.add(new IntroduceStaticTypeCastRule());
+        normalization.add(new IntroduceStaticTypeCastForInsertRule());
         normalization.add(new IntroduceDynamicTypeCastRule());
         normalization.add(new IntroduceEnforcedListTypeRule());
         normalization.add(new ConstantFoldingRule());
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java
index b0ce342..ae0a842 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -112,7 +112,7 @@
             AqlTypeTraitProvider.INSTANCE, AqlBinaryBooleanInspectorImpl.FACTORY, AqlBinaryIntegerInspector.FACTORY,
             AqlPrinterFactoryProvider.INSTANCE, AqlNullWriterFactory.INSTANCE, null,
             new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(AqlLogicalExpressionJobGen.INSTANCE),
-            AqlExpressionTypeComputer.INSTANCE, AqlNullableTypeComputer.INSTANCE, null, null, null,
+            AqlExpressionTypeComputer.INSTANCE, AqlNullableTypeComputer.INSTANCE, null, null, null, null,
             GlobalConfig.DEFAULT_FRAME_SIZE, null);
 
     private static final IOperatorSchema[] _emptySchemas = new IOperatorSchema[] {};
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java
index 2e8d530..a07583f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java
@@ -20,6 +20,7 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
+import edu.uci.ics.asterix.om.typecomputer.base.TypeComputerUtilities;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.optimizer.rules.typecast.StaticTypeCastUtil;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -27,7 +28,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractAssignOperator;
@@ -53,26 +53,27 @@
             throws AlgebricksException {
         if (context.checkIfInDontApplySet(this, opRef.getValue()))
             return false;
-        AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         context.addToDontApplySet(this, opRef.getValue());
-        boolean changed = false;
 
         /**
          * rewrite list constructor types for list constructor functions
          */
-        if (op1.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-            AbstractAssignOperator assignOp = (AbstractAssignOperator) op1;
-            List<Mutable<ILogicalExpression>> expressions = assignOp.getExpressions();
-            IVariableTypeEnvironment env = assignOp.computeOutputTypeEnvironment(context);
-            changed = rewriteExpressions(expressions, env);
+        List<Mutable<ILogicalExpression>> expressions;
+        switch (op.getOperatorTag()) {
+            case ASSIGN:
+                AbstractAssignOperator assignOp = (AbstractAssignOperator) op;
+                expressions = assignOp.getExpressions();
+                break;
+            case UNNEST:
+                AbstractUnnestOperator unnestOp = (AbstractUnnestOperator) op;
+                expressions = Collections.singletonList(unnestOp.getExpressionRef());
+                break;
+            default:
+                return false;
         }
-        if (op1.getOperatorTag() == LogicalOperatorTag.UNNEST) {
-            AbstractUnnestOperator unnestOp = (AbstractUnnestOperator) op1;
-            List<Mutable<ILogicalExpression>> expressions = Collections.singletonList(unnestOp.getExpressionRef());
-            IVariableTypeEnvironment env = unnestOp.computeOutputTypeEnvironment(context);
-            changed = rewriteExpressions(expressions, env);
-        }
-        return changed;
+        IVariableTypeEnvironment env = op.computeOutputTypeEnvironment(context);
+        return rewriteExpressions(expressions, env);
     }
 
     private boolean rewriteExpressions(List<Mutable<ILogicalExpression>> expressions, IVariableTypeEnvironment env)
@@ -83,7 +84,10 @@
             if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) expr;
                 IAType exprType = (IAType) env.getType(argFuncExpr);
-                changed = changed || StaticTypeCastUtil.rewriteListExpr(argFuncExpr, exprType, exprType, env);
+                if (StaticTypeCastUtil.rewriteListExpr(argFuncExpr, exprType, exprType, env)) {
+                    TypeComputerUtilities.resetRequiredAndInputTypes(argFuncExpr);
+                    changed = true;
+                }
             }
         }
         return changed;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastForInsertRule.java
similarity index 98%
rename from asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
rename to asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastForInsertRule.java
index f42782b..850f01d 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastForInsertRule.java
@@ -60,7 +60,7 @@
  * ConstantFoldingRule uses ARecordSerializerDeserializer which seems to have
  * some problem.
  */
-public class IntroduceStaticTypeCastRule implements IAlgebraicRewriteRule {
+public class IntroduceStaticTypeCastForInsertRule implements IAlgebraicRewriteRule {
 
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java
new file mode 100644
index 0000000..984f4dd
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.optimizer.rules;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * This rule introduces a unnest operator for the collection-to-sequence function (if the input to the function is a collection).
+ * 
+ * @author yingyib
+ */
+public class IntroduceUnnestForCollectionToSequenceRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+            return false;
+        }
+        AssignOperator assign = (AssignOperator) op;
+        List<Mutable<ILogicalExpression>> exprs = assign.getExpressions();
+        if (exprs.size() != 1) {
+            return false;
+        }
+        ILogicalExpression expr = exprs.get(0).getValue();
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression func = (AbstractFunctionCallExpression) expr;
+        if (func.getFunctionIdentifier() != AsterixBuiltinFunctions.COLLECTION_TO_SEQUENCE) {
+            return false;
+        }
+
+        IVariableTypeEnvironment env = assign.computeInputTypeEnvironment(context);
+        ILogicalExpression argExpr = func.getArguments().get(0).getValue();
+        IAType outerExprType = (IAType) env.getType(expr);
+        IAType innerExprType = (IAType) env.getType(argExpr);
+        if (outerExprType.equals(innerExprType)) {
+            /** nothing is changed with the collection-to-sequence function, remove the collection-sequence function call */
+            assign.getExpressions().set(0, new MutableObject<ILogicalExpression>(argExpr));
+            return true;
+        }
+        /** change the assign operator to an unnest operator */
+        LogicalVariable var = assign.getVariables().get(0);
+        @SuppressWarnings("unchecked")
+        UnnestOperator unnest = new UnnestOperator(var, new MutableObject<ILogicalExpression>(
+                new UnnestingFunctionCallExpression(
+                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+                        new MutableObject<ILogicalExpression>(argExpr))));
+        unnest.getInputs().addAll(assign.getInputs());
+        opRef.setValue(unnest);
+        context.computeAndSetTypeEnvironmentForOperator(unnest);
+        return true;
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
index 30ed8e7..f277d4b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
@@ -7,9 +7,9 @@
 
 import edu.uci.ics.asterix.algebra.operators.CommitOperator;
 import edu.uci.ics.asterix.algebra.operators.physical.CommitPOperator;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index d6e0f42..e5d6bfc 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
@@ -47,6 +47,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 
 /**
@@ -98,7 +99,7 @@
                 if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                     AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
                     IAType exprType = (IAType) env.getType(argFuncExpr);
-                    changed = changed || rewriteListExpr(argFuncExpr, exprType, exprType, env);
+                    changed = rewriteListExpr(argFuncExpr, exprType, exprType, env) || changed;
                 }
             }
             return changed;
@@ -163,7 +164,7 @@
                 }
             }
             if (!compatible(reqType, inputType)) {
-                throw new AlgebricksException("type mistmach, requred: " + reqType.toString() + " actual: "
+                throw new AlgebricksException("type mismatch, required: " + reqType.toString() + " actual: "
                         + inputType.toString());
             }
             return changed;
@@ -188,8 +189,12 @@
         // if already rewritten, the required type is not null
         if (TypeComputerUtilities.getRequiredType(funcExpr) != null)
             return false;
-        TypeComputerUtilities.setRequiredAndInputTypes(funcExpr, requiredRecordType, inputRecordType);
-        return staticRecordTypeCast(funcExpr, requiredRecordType, inputRecordType, env);
+        boolean casted = staticRecordTypeCast(funcExpr, requiredRecordType, inputRecordType, env);
+        if (casted) {
+            //enforce the required type if it is statically casted
+            TypeComputerUtilities.setRequiredAndInputTypes(funcExpr, requiredRecordType, inputRecordType);
+        }
+        return casted;
     }
 
     /**
@@ -215,20 +220,21 @@
 
         IAType itemType = requiredListType.getItemType();
         IAType inputItemType = inputListType.getItemType();
+        boolean changed = false;
         for (int j = 0; j < args.size(); j++) {
             ILogicalExpression arg = args.get(j).getValue();
-            if (arg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) arg;
-                IAType currentItemType = (IAType) env.getType(argFunc);
-                if (inputItemType == null || inputItemType == BuiltinType.ANY) {
-                    currentItemType = (IAType) env.getType(argFunc);
-                    rewriteFuncExpr(argFunc, itemType, currentItemType, env);
-                } else {
-                    rewriteFuncExpr(argFunc, itemType, inputItemType, env);
-                }
+            IAType currentItemType = (inputItemType == null || inputItemType == BuiltinType.ANY) ? (IAType) env.getType(arg) : inputItemType;
+            switch (arg.getExpressionTag()) {
+                case FUNCTION_CALL:
+                    ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) arg;
+                    changed = rewriteFuncExpr(argFunc, itemType, currentItemType, env) || changed;
+                    break;
+                case VARIABLE:
+                    changed = injectCastToRelaxType(args.get(j), currentItemType, env) || changed;
+                    break;
             }
         }
-        return true;
+        return changed;
     }
 
     /**
@@ -422,67 +428,68 @@
             if (openFields[i]) {
                 arguments.add(originalArguments.get(2 * i));
                 Mutable<ILogicalExpression> expRef = originalArguments.get(2 * i + 1);
-                ILogicalExpression argExpr = expRef.getValue();
-                List<LogicalVariable> parameterVars = new ArrayList<LogicalVariable>();
-                argExpr.getUsedVariables(parameterVars);
-                // we need to handle open fields recursively by their default
-                // types
-                // for list, their item type is any
-                // for record, their
-                boolean castInjected = false;
-                if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
-                        || argExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                    IAType reqFieldType = inputFieldTypes[i];
-                    // do not enforce nested type in the case of no-used variables
-                    if (inputFieldTypes[i].getTypeTag() == ATypeTag.RECORD) {
-                        reqFieldType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
-                        if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
-                            //inject dynamic type casting
-                            injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_RECORD),
-                                    reqFieldType, inputFieldTypes[i], expRef, argExpr);
-                            castInjected = true;
-                        }
-                    }
-                    if (inputFieldTypes[i].getTypeTag() == ATypeTag.ORDEREDLIST) {
-                        reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
-                        if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
-                            //inject dynamic type casting
-                            injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_LIST),
-                                    reqFieldType, inputFieldTypes[i], expRef, argExpr);
-                            castInjected = true;
-                        }
-                    }
-                    if (inputFieldTypes[i].getTypeTag() == ATypeTag.UNORDEREDLIST) {
-                        reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
-                        if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
-                            //inject dynamic type casting
-                            injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_LIST),
-                                    reqFieldType, inputFieldTypes[i], expRef, argExpr);
-                            castInjected = true;
-                        }
-                    }
-                    if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                        //recursively rewrite function arguments
-                        if (TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null
-                                && reqFieldType != null) {
-                            if (castInjected) {
-                                //rewrite the arg expression inside the dynamic cast
-                                ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
-                                rewriteFuncExpr(argFunc, inputFieldTypes[i], inputFieldTypes[i], env);
-                            } else {
-                                //rewrite arg
-                                ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
-                                rewriteFuncExpr(argFunc, reqFieldType, inputFieldTypes[i], env);
-                            }
-                        }
-                    }
-                }
+                injectCastToRelaxType(expRef, inputFieldTypes[i], env);
                 arguments.add(expRef);
             }
         }
         return true;
     }
 
+    private static boolean injectCastToRelaxType(Mutable<ILogicalExpression> expRef, IAType inputFieldType,
+            IVariableTypeEnvironment env) throws AlgebricksException {
+        ILogicalExpression argExpr = expRef.getValue();
+        List<LogicalVariable> parameterVars = new ArrayList<LogicalVariable>();
+        argExpr.getUsedVariables(parameterVars);
+        // we need to handle open fields recursively by their default
+        // types
+        // for list, their item type is any
+        // for record, their
+        boolean castInjected = false;
+        if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
+                || argExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+            IAType reqFieldType = inputFieldType;
+            FunctionIdentifier fi = null;
+            // do not enforce nested type in the case of no-used variables
+            switch (inputFieldType.getTypeTag()) {
+                case RECORD:
+                    reqFieldType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+                    fi = AsterixBuiltinFunctions.CAST_RECORD;
+                    break;
+                case ORDEREDLIST:
+                    reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+                    fi = AsterixBuiltinFunctions.CAST_LIST;
+                    break;
+                case UNORDEREDLIST:
+                    reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+                    fi = AsterixBuiltinFunctions.CAST_LIST;
+            }
+            if (fi != null
+                    && ! inputFieldType.equals(reqFieldType)
+                    && parameterVars.size() > 0) {
+                //inject dynamic type casting
+                injectCastFunction(FunctionUtils.getFunctionInfo(fi),
+                        reqFieldType, inputFieldType, expRef, argExpr);
+                castInjected = true;
+            }
+            if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                //recursively rewrite function arguments
+                if (TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null
+                        && reqFieldType != null) {
+                    if (castInjected) {
+                        //rewrite the arg expression inside the dynamic cast
+                        ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
+                        rewriteFuncExpr(argFunc, inputFieldType, inputFieldType, env);
+                    } else {
+                        //rewrite arg
+                        ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
+                        rewriteFuncExpr(argFunc, reqFieldType, inputFieldType, env);
+                    }
+                }
+            }
+        }
+        return castInjected;
+    }
+
     /**
      * Inject a dynamic cast function wrapping an existing expression
      * 
@@ -512,7 +519,7 @@
      *            the required type
      * @param inputType
      *            the input type
-     * @return true if the two types are compatiable; false otherwise
+     * @return true if the two types are compatible; false otherwise
      */
     public static boolean compatible(IAType reqType, IAType inputType) {
         if (reqType.getTypeTag() == ATypeTag.ANY || inputType.getTypeTag() == ATypeTag.ANY) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
index 9fe5750..792ff25 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.asterix.metadata.entities.AsterixBuiltinTypeMap;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 
 /**
@@ -41,6 +42,12 @@
     protected static final Map<String, BuiltinType> builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
 
     public void validateOperation(Dataverse defaultDataverse, Statement stmt) throws AsterixException {
+
+        if (AsterixClusterProperties.INSTANCE.getState().equals(AsterixClusterProperties.State.UNUSABLE)) {
+            throw new AsterixException(" Asterix Cluster is in " + AsterixClusterProperties.State.UNUSABLE + " state."
+                    + "\n One or more Node Controllers have left.\n");
+        }
+
         boolean invalidOperation = false;
         String message = null;
         String dataverse = defaultDataverse != null ? defaultDataverse.getDataverseName() : null;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
index e07c1d0..fc4bb03d7 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -71,7 +71,6 @@
 import edu.uci.ics.asterix.aql.expression.WriteStatement;
 import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
 import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -92,10 +91,10 @@
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions.FunctionNamespace;
 import edu.uci.ics.asterix.om.functions.AsterixFunctionInfo;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.asterix.runtime.formats.FormatUtils;
 import edu.uci.ics.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -188,13 +187,13 @@
         ILogicalOperator topOp = p.first;
         ProjectOperator project = (ProjectOperator) topOp;
         LogicalVariable resVar = project.getVariables().get(0);
+
         if (outputDatasetName == null) {
             FileSplit outputFileSplit = metadataProvider.getOutputFile();
             if (outputFileSplit == null) {
                 outputFileSplit = getDefaultOutputFileLocation();
             }
             metadataProvider.setOutputFile(outputFileSplit);
-            String resultNodeName = outputFileSplit.getNodeName();
 
             List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
             writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
@@ -203,6 +202,20 @@
             topOp = new DistributeResultOperator(writeExprList, sink);
             topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
         } else {
+            /** add the collection-to-sequence right before the final project, because dataset only accept non-collection records */
+            LogicalVariable seqVar = context.newVar();
+            @SuppressWarnings("unchecked")
+            /** This assign adds a marker function collection-to-sequence: if the input is a singleton collection, unnest it; otherwise do nothing. */
+            AssignOperator assignCollectionToSequence = new AssignOperator(seqVar,
+                    new MutableObject<ILogicalExpression>(
+                            new ScalarFunctionCallExpression(AsterixBuiltinFunctions
+                                    .getAsterixFunctionInfo(AsterixBuiltinFunctions.COLLECTION_TO_SEQUENCE),
+                                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)))));
+            assignCollectionToSequence.getInputs().add(
+                    new MutableObject<ILogicalOperator>(project.getInputs().get(0).getValue()));
+            project.getInputs().get(0).setValue(assignCollectionToSequence);
+            project.getVariables().set(0, seqVar);
+            resVar = seqVar;
 
             AqlDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
                     stmt.getDatasetName());
@@ -226,7 +239,6 @@
             }
             AssignOperator assign = new AssignOperator(vars, exprs);
             assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
-
             Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
                     resVar));
             ILogicalOperator leafOperator = null;
@@ -507,14 +519,10 @@
         if (builtinAquafi != null) {
             fi = builtinAquafi;
         } else {
-            fi = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(), functionName, arity);
+            fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, functionName, arity);
             afi = AsterixBuiltinFunctions.lookupFunction(fi);
             if (afi == null) {
-                fi = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(), functionName, arity);
-                afi = AsterixBuiltinFunctions.lookupFunction(fi);
-                if (afi == null) {
-                    return null;
-                }
+                return null;
             }
         }
         if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
index 92add99..0f636a9 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
@@ -79,6 +79,7 @@
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.FileSplitDataSink;
@@ -89,12 +90,10 @@
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions.FunctionNamespace;
 import edu.uci.ics.asterix.om.functions.AsterixFunctionInfo;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
@@ -465,14 +464,10 @@
         if (builtinAquafi != null) {
             fi = builtinAquafi;
         } else {
-            fi = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(), signature.getName());
+            fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, signature.getName());
             FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
             if (builtinAsterixFi != null) {
                 fi = builtinAsterixFi;
-            } else {
-                fi = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(), signature.getName());
-                builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
-                fi = builtinAsterixFi;
             }
         }
         AbstractFunctionCallExpression f;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
index a91f2eb..49e2ed6 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
@@ -35,6 +35,7 @@
 import edu.uci.ics.asterix.aql.expression.WhereClause;
 import edu.uci.ics.asterix.aql.literal.StringLiteral;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
@@ -496,7 +497,7 @@
             arguments.add(argumentLiteral);
 
             CallExpr callExpression = new CallExpr(new FunctionSignature(
-                    AsterixBuiltinFunctions.FunctionNamespace.ASTERIX_PUBLIC.name(), "dataset", 1), arguments);
+                    FunctionConstants.ASTERIX_NS, "dataset", 1), arguments);
             List<Clause> clauseList = new ArrayList<Clause>();
             Clause forClause = new ForClause(var, callExpression);
             clauseList.add(forClause);
diff --git a/asterix-app/data/fbu.adm b/asterix-app/data/fbu.adm
new file mode 100644
index 0000000..c93c262
--- /dev/null
+++ b/asterix-app/data/fbu.adm
@@ -0,0 +1,1000 @@
+{"id":11381089,"id-copy":11381089,"alias":"Earlene","name":"EarleneAmmons","user-since":datetime("2010-03-24T05:25:35"),"user-since-copy":datetime("2010-03-24T05:25:35"),"friend-ids":{{25392364,36996951,16110083,9799716,22893553,28551996,7706432,14225386,15633254,39395931,46707062,37226919,8532306,3765988,20939685,31136325,45222021,15355741,8760941,12045616,6890610,13560532,44914868,37744233}},"employment":[{"organization-name":"Roundhex","start-date":date("2000-06-10")}]}
+{"id":10495420,"id-copy":10495420,"alias":"Wendy","name":"WendyMcloskey","user-since":datetime("2011-04-26T23:38:24"),"user-since-copy":datetime("2011-04-26T23:38:24"),"friend-ids":{{16762653,46262691,12313140,20481262,347993,23105127,1680519,20880265,45611347,21907223,46615281,17188244,44019800,46943250,28647738,16792673,29406270,42714079}},"employment":[{"organization-name":"Qvohouse","start-date":date("2008-08-27")}]}
+{"id":10957867,"id-copy":10957867,"alias":"Zach","name":"ZachOppenheimer","user-since":datetime("2012-01-01T14:40:11"),"user-since-copy":datetime("2012-01-01T14:40:11"),"friend-ids":{{27759480,2112389,8560433,10052851,37714587,16717012,36648956,44803993,36030695,5359496,32302980,27143894,19287706}},"employment":[{"organization-name":"Tanzumbam","start-date":date("2003-05-14"),"end-date":date("2004-02-23")}]}
+{"id":9988417,"id-copy":9988417,"alias":"Coline","name":"ColineLane","user-since":datetime("2010-01-01T00:12:39"),"user-since-copy":datetime("2010-01-01T00:12:39"),"friend-ids":{{17656229,42804152}},"employment":[{"organization-name":"Fax-fax","start-date":date("2012-05-01")}]}
+{"id":10272571,"id-copy":10272571,"alias":"Jarrett","name":"JarrettGoldvogel","user-since":datetime("2010-04-28T23:24:22"),"user-since-copy":datetime("2010-04-28T23:24:22"),"friend-ids":{{47024505,36647273,32152567,28239957,11739703,47515825,17408763,41224279,41487670,43339913}},"employment":[{"organization-name":"Transhigh","start-date":date("2004-04-06"),"end-date":date("2010-02-14")}]}
+{"id":11307946,"id-copy":11307946,"alias":"Helga","name":"HelgaStough","user-since":datetime("2007-01-12T21:50:11"),"user-since-copy":datetime("2007-01-12T21:50:11"),"friend-ids":{{22768365}},"employment":[{"organization-name":"subtam","start-date":date("2007-01-04"),"end-date":date("2009-06-25")}]}
+{"id":11061631,"id-copy":11061631,"alias":"Maxene","name":"MaxeneKellogg","user-since":datetime("2005-11-13T01:09:31"),"user-since-copy":datetime("2005-11-13T01:09:31"),"friend-ids":{{31578394,39466620,35741359,14244925,3000582,39031643,5008430,18315325,30440631,37868108,12014032,32314102,42887702,1853960,28022174,2024670,38864358,42073112,16259942,34693959,25315399,37475597,33599283}},"employment":[{"organization-name":"Unijobam","start-date":date("2008-05-13")}]}
+{"id":10874791,"id-copy":10874791,"alias":"Haydee","name":"HaydeeGarratt","user-since":datetime("2007-04-14T00:19:00"),"user-since-copy":datetime("2007-04-14T00:19:00"),"friend-ids":{{12247794,10306863,33161811,43877113,37745696}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2008-03-07"),"end-date":date("2011-12-27")}]}
+{"id":11570326,"id-copy":11570326,"alias":"Linden","name":"LindenFilby","user-since":datetime("2007-08-16T03:11:11"),"user-since-copy":datetime("2007-08-16T03:11:11"),"friend-ids":{{6549689,15243636,3147666}},"employment":[{"organization-name":"Solfix","start-date":date("2010-02-23"),"end-date":date("2010-04-22")}]}
+{"id":10498285,"id-copy":10498285,"alias":"Kiley","name":"KileyBridger","user-since":datetime("2006-05-14T21:55:34"),"user-since-copy":datetime("2006-05-14T21:55:34"),"friend-ids":{{38780484,46190003,905670,35609390,46621151,5099226,24328595,16340411,13326485,13872400,35896828,9196151,8525875,7461206,28379538,46461267,45270205,35718577,5310596,7080391}},"employment":[{"organization-name":"Newcom","start-date":date("2009-11-11"),"end-date":date("2009-06-23")}]}
+{"id":9629395,"id-copy":9629395,"alias":"Julius","name":"JuliusWire","user-since":datetime("2008-03-22T13:36:24"),"user-since-copy":datetime("2008-03-22T13:36:24"),"friend-ids":{{}},"employment":[{"organization-name":"Tranzap","start-date":date("2006-11-19")}]}
+{"id":11447332,"id-copy":11447332,"alias":"Sherisse","name":"SherisseMaugham","user-since":datetime("2012-02-09T14:21:08"),"user-since-copy":datetime("2012-02-09T14:21:08"),"friend-ids":{{}},"employment":[{"organization-name":"Tripplelane","start-date":date("2011-09-16")}]}
+{"id":10179538,"id-copy":10179538,"alias":"Orlando","name":"OrlandoBaxter","user-since":datetime("2006-02-06T08:33:07"),"user-since-copy":datetime("2006-02-06T08:33:07"),"friend-ids":{{6233497,33888281,44259464,19279042,22534429,13084190,38886041,41675566,3155617}},"employment":[{"organization-name":"Ontohothex","start-date":date("2009-07-06")}]}
+{"id":10001080,"id-copy":10001080,"alias":"Garrett","name":"GarrettBode","user-since":datetime("2005-10-25T18:07:35"),"user-since-copy":datetime("2005-10-25T18:07:35"),"friend-ids":{{35858744,16426061,11473961,4769664,29038930,33070686,46271872,42593454,36202882,46642640,22243678,20222041,29014540,7389258,7172909,12787979,146736,21081030,21615179,2936936,44934891}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2007-06-24")}]}
+{"id":11675221,"id-copy":11675221,"alias":"Calanthe","name":"CalantheGearhart","user-since":datetime("2007-06-08T02:44:20"),"user-since-copy":datetime("2007-06-08T02:44:20"),"friend-ids":{{19185575}},"employment":[{"organization-name":"Vivaace","start-date":date("2010-05-21")}]}
+{"id":11140213,"id-copy":11140213,"alias":"Montgomery","name":"MontgomeryWhittier","user-since":datetime("2007-06-19T17:46:13"),"user-since-copy":datetime("2007-06-19T17:46:13"),"friend-ids":{{32831460,6030454,30437362,21866470,17388602,40815157,20000967,47555494,5818137,40634742,21692148,2365521,33290069,46471164,9192561,35768343,7552168,3577338,5346012,31129868}},"employment":[{"organization-name":"Y-geohex","start-date":date("2008-02-24")}]}
+{"id":11954992,"id-copy":11954992,"alias":"Caitlin","name":"CaitlinLangston","user-since":datetime("2007-01-02T01:50:34"),"user-since-copy":datetime("2007-01-02T01:50:34"),"friend-ids":{{23355687,22474136,28513847,32515387,44041844,33706721,10874992,36341753,34431157,16146113,15462591,18188151,29554174,44940738,25888018,42795884,14382632,12734889,11724519,15830341,25725320,37580394,24124411,47984339}},"employment":[{"organization-name":"Kanelectrics","start-date":date("2010-05-26"),"end-date":date("2010-03-28")}]}
+{"id":9510451,"id-copy":9510451,"alias":"Chuck","name":"ChuckFinck","user-since":datetime("2011-09-10T08:27:31"),"user-since-copy":datetime("2011-09-10T08:27:31"),"friend-ids":{{5559039,8997599,8311284,20478562,13734713,21511695,30393493}},"employment":[{"organization-name":"Inchdox","start-date":date("2001-10-12")}]}
+{"id":11068231,"id-copy":11068231,"alias":"Dinah","name":"DinahSwink","user-since":datetime("2012-05-02T04:24:33"),"user-since-copy":datetime("2012-05-02T04:24:33"),"friend-ids":{{31542440,17451543,32642661,27867264,32718667,43042567,7921827}},"employment":[{"organization-name":"highfax","start-date":date("2003-04-10"),"end-date":date("2003-10-03")}]}
+{"id":10361965,"id-copy":10361965,"alias":"Arlen","name":"ArlenFlick","user-since":datetime("2011-07-14T18:38:37"),"user-since-copy":datetime("2011-07-14T18:38:37"),"friend-ids":{{34249140,2887282,47622716,3897801,33692288,14374380,14183995,41311739,6378075,17721901,20807501,8908974,41080464,26497672}},"employment":[{"organization-name":"Medflex","start-date":date("2008-05-18"),"end-date":date("2011-09-18")}]}
+{"id":10423588,"id-copy":10423588,"alias":"Shirlene","name":"ShirleneRuch","user-since":datetime("2006-04-09T05:52:24"),"user-since-copy":datetime("2006-04-09T05:52:24"),"friend-ids":{{15418780,12724265,27282306,13592995,24753166,32824252,40619106,27563604,12337625,45387219,27749581,44912564,37470078,19663516}},"employment":[{"organization-name":"Newphase","start-date":date("2003-06-17")}]}
+{"id":11951098,"id-copy":11951098,"alias":"Tera","name":"TeraByers","user-since":datetime("2012-08-03T19:41:26"),"user-since-copy":datetime("2012-08-03T19:41:26"),"friend-ids":{{15537238,13699967,10587728,23542817,12703626,25024772,19223339,5547239,42576945,27351017,22726496,25268071,4361323,24631578,38669047,44781738,34646381}},"employment":[{"organization-name":"Sublamdox","start-date":date("2008-01-04"),"end-date":date("2011-01-14")}]}
+{"id":9594523,"id-copy":9594523,"alias":"Tam","name":"TamWillcox","user-since":datetime("2011-12-23T11:41:58"),"user-since-copy":datetime("2011-12-23T11:41:58"),"friend-ids":{{27383896,20745988,10063024,8241427,40299998,32408463,25171835,22380586,15344194,25951348,28733234,45421004,2273747,2229862,6241144,6704115,8659430,47431991,47929530,24393021}},"employment":[{"organization-name":"Keytech","start-date":date("2001-07-27")}]}
+{"id":9478720,"id-copy":9478720,"alias":"Angelia","name":"AngeliaKettlewell","user-since":datetime("2005-05-27T06:29:30"),"user-since-copy":datetime("2005-05-27T06:29:30"),"friend-ids":{{42556433,20033025,38112512,19420757,31822717,7116081,39544900,19203395,46787205,32303456,4509345,45558040,42616291,6929369,9272653,37459048,37113569,38942369,47741031,46761451,14163845}},"employment":[{"organization-name":"Alphadax","start-date":date("2012-03-28"),"end-date":date("2012-03-04")}]}
+{"id":9142198,"id-copy":9142198,"alias":"Sherry","name":"SherryFea","user-since":datetime("2011-03-28T23:09:22"),"user-since-copy":datetime("2011-03-28T23:09:22"),"friend-ids":{{6835080,34471872,30942941,34858577,5996593,47293442,43097072,44809621,33969893,26410931,6628186,29944391,35957320,20326929,40284077,11681583,43878314,40265961,16871274,28406169,1349311}},"employment":[{"organization-name":"Mathtech","start-date":date("2004-07-28")}]}
+{"id":9313492,"id-copy":9313492,"alias":"Tera","name":"TeraWolfe","user-since":datetime("2010-12-20T12:47:25"),"user-since-copy":datetime("2010-12-20T12:47:25"),"friend-ids":{{45424983,18345704,14849759,31638064,38670515,48015953,36114769}},"employment":[{"organization-name":"Redelectronics","start-date":date("2001-04-26"),"end-date":date("2004-12-06")}]}
+{"id":10307032,"id-copy":10307032,"alias":"Quentin","name":"QuentinSauter","user-since":datetime("2012-07-11T07:16:43"),"user-since-copy":datetime("2012-07-11T07:16:43"),"friend-ids":{{1926278,42211794,1508832,14973540,6721046,28872485,5047722,7805271,31508326,20891455,38735410,13190567,18209753,44468536,34640135,47290587,25576626}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2012-02-13")}]}
+{"id":10733617,"id-copy":10733617,"alias":"Leonardo","name":"LeonardoKight","user-since":datetime("2008-10-20T17:30:29"),"user-since-copy":datetime("2008-10-20T17:30:29"),"friend-ids":{{39687903,7235506,34696496,25995345,18435380,47473591,15710408,44232442,39520147,36384026,25160887,245860,1195579,4587411,536916,47052672,33953823,13203710}},"employment":[{"organization-name":"tresline","start-date":date("2007-07-12"),"end-date":date("2010-03-16")}]}
+{"id":10394488,"id-copy":10394488,"alias":"Oswald","name":"OswaldRay","user-since":datetime("2006-02-12T17:39:23"),"user-since-copy":datetime("2006-02-12T17:39:23"),"friend-ids":{{14370372,14174983,7749259,39375970,1755409,9056913}},"employment":[{"organization-name":"Canline","start-date":date("2011-12-04"),"end-date":date("2011-06-08")}]}
+{"id":11081539,"id-copy":11081539,"alias":"Haidee","name":"HaideeStyle","user-since":datetime("2012-06-13T11:37:34"),"user-since-copy":datetime("2012-06-13T11:37:34"),"friend-ids":{{}},"employment":[{"organization-name":"Statcode","start-date":date("2001-03-05"),"end-date":date("2003-11-17")}]}
+{"id":9211711,"id-copy":9211711,"alias":"Seraphina","name":"SeraphinaFlanders","user-since":datetime("2009-05-19T18:39:15"),"user-since-copy":datetime("2009-05-19T18:39:15"),"friend-ids":{{34432294,10796959,46386746,32318131,10393677,12832313,34490791,6187782,46595448,30591963,35530646,22485004,18950892,19762388,19181134,13928403,22513246,24969298}},"employment":[{"organization-name":"Ganjastrip","start-date":date("2008-05-14"),"end-date":date("2009-06-17")}]}
+{"id":10595164,"id-copy":10595164,"alias":"Jerome","name":"JeromeLacon","user-since":datetime("2009-09-24T09:47:36"),"user-since-copy":datetime("2009-09-24T09:47:36"),"friend-ids":{{31538601}},"employment":[{"organization-name":"Ransaofan","start-date":date("2012-07-26")}]}
+{"id":10284583,"id-copy":10284583,"alias":"Salal","name":"SalalButterfill","user-since":datetime("2011-02-05T13:39:36"),"user-since-copy":datetime("2011-02-05T13:39:36"),"friend-ids":{{}},"employment":[{"organization-name":"Basecone","start-date":date("2008-08-10"),"end-date":date("2011-05-02")}]}
+{"id":11425216,"id-copy":11425216,"alias":"Levi","name":"LeviEiford","user-since":datetime("2010-04-10T23:37:26"),"user-since-copy":datetime("2010-04-10T23:37:26"),"friend-ids":{{39348801,15029457,33995161,27782571,16712478,28987111}},"employment":[{"organization-name":"Zuncan","start-date":date("2002-08-12")}]}
+{"id":11103856,"id-copy":11103856,"alias":"Dennise","name":"DenniseGarland","user-since":datetime("2008-10-19T11:09:14"),"user-since-copy":datetime("2008-10-19T11:09:14"),"friend-ids":{{2613052,4777379,29911213,30822813,44182985,803163,32630608,7433428,43625503,19274272,20950244,21434389,44059623,40416129,47937344,12392360}},"employment":[{"organization-name":"whitestreet","start-date":date("2005-04-10"),"end-date":date("2005-07-26")}]}
+{"id":9440818,"id-copy":9440818,"alias":"Poppy","name":"PoppyBoyer","user-since":datetime("2007-06-09T08:15:05"),"user-since-copy":datetime("2007-06-09T08:15:05"),"friend-ids":{{10721272,26882431,45774996,44725231,34694934,28877797,12922671,16078039,43902220,27311426,34146150,39285332,7343219,17482231,15496713,12439079,18097780,30046636,16951144,27968612}},"employment":[{"organization-name":"Sublamdox","start-date":date("2006-11-17")}]}
+{"id":9878209,"id-copy":9878209,"alias":"Duana","name":"DuanaGettemy","user-since":datetime("2007-03-05T19:06:27"),"user-since-copy":datetime("2007-03-05T19:06:27"),"friend-ids":{{5530171,22409344,22742046,14418589,27149252}},"employment":[{"organization-name":"Ontotanin","start-date":date("2012-08-07")}]}
+{"id":10417531,"id-copy":10417531,"alias":"Eileen","name":"EileenCrissman","user-since":datetime("2009-10-13T21:36:38"),"user-since-copy":datetime("2009-10-13T21:36:38"),"friend-ids":{{911579,3590209,15646563,31960066,14495212,44915460,42713118,1962949,44935091,6578467,21896024,41455809,25543039,28884330,44289305,15569750,32580470,46016098,9828368}},"employment":[{"organization-name":"Plexlane","start-date":date("2003-06-11"),"end-date":date("2005-10-02")}]}
+{"id":10674199,"id-copy":10674199,"alias":"Dorothy","name":"DorothyPritchard","user-since":datetime("2007-09-19T04:32:05"),"user-since-copy":datetime("2007-09-19T04:32:05"),"friend-ids":{{11239155,14468542,8244419,30563447,2235193,33015958,11941749,22198664,41531114,11614864,43486312,11394784,46038310,8248070,12346192}},"employment":[{"organization-name":"Basecone","start-date":date("2000-10-03")}]}
+{"id":9503761,"id-copy":9503761,"alias":"Demelza","name":"DemelzaLaw","user-since":datetime("2010-12-17T06:40:19"),"user-since-copy":datetime("2010-12-17T06:40:19"),"friend-ids":{{34126746,5537488,609154,35877951,36237612}},"employment":[{"organization-name":"Ontotanin","start-date":date("2005-10-22")}]}
+{"id":9216376,"id-copy":9216376,"alias":"Stanford","name":"StanfordBurney","user-since":datetime("2010-04-24T23:03:06"),"user-since-copy":datetime("2010-04-24T23:03:06"),"friend-ids":{{15567770,24839882,163708,45725879,43621238,27363995,46782727,21660511,37585197,17426559,47247057,41831246,23944363,1608826,25831838,41140458,37108898,19739056,7475981,17807472,3126856,40257768,44873566}},"employment":[{"organization-name":"U-ron","start-date":date("2004-10-04")}]}
+{"id":11188879,"id-copy":11188879,"alias":"Corrie","name":"CorrieOsterwise","user-since":datetime("2011-01-20T21:11:19"),"user-since-copy":datetime("2011-01-20T21:11:19"),"friend-ids":{{47499393,41394452,27330253,14958477,14558879,47694640,28440147,3437209,40720108,26390443}},"employment":[{"organization-name":"Hexsanhex","start-date":date("2007-08-28")}]}
+{"id":11129635,"id-copy":11129635,"alias":"Porter","name":"PorterRohtin","user-since":datetime("2005-08-07T05:18:16"),"user-since-copy":datetime("2005-08-07T05:18:16"),"friend-ids":{{15192554,37509296,35638203,5517199,3781940,43497242,28477558,4325184,34919156,18037278,36486191,13966437,16629611,40623060}},"employment":[{"organization-name":"Zimcone","start-date":date("2005-07-13")}]}
+{"id":11001610,"id-copy":11001610,"alias":"Keven","name":"KevenWildman","user-since":datetime("2006-09-07T02:21:33"),"user-since-copy":datetime("2006-09-07T02:21:33"),"friend-ids":{{14316856,4291050}},"employment":[{"organization-name":"Ganjatax","start-date":date("2012-06-20"),"end-date":date("2012-06-09")}]}
+{"id":9890854,"id-copy":9890854,"alias":"Linwood","name":"LinwoodBrown","user-since":datetime("2005-09-09T12:38:00"),"user-since-copy":datetime("2005-09-09T12:38:00"),"friend-ids":{{13728190,31562633,3437344,13841675,38528685}},"employment":[{"organization-name":"Hexviafind","start-date":date("2006-05-08"),"end-date":date("2009-08-26")}]}
+{"id":9326218,"id-copy":9326218,"alias":"Lindsay","name":"LindsayPaynter","user-since":datetime("2011-08-27T00:03:13"),"user-since-copy":datetime("2011-08-27T00:03:13"),"friend-ids":{{3006430,25941368,46866627,21404266,35141764,14931901}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2008-04-06"),"end-date":date("2008-03-02")}]}
+{"id":10834579,"id-copy":10834579,"alias":"Penni","name":"PenniBlunt","user-since":datetime("2010-05-20T20:29:16"),"user-since-copy":datetime("2010-05-20T20:29:16"),"friend-ids":{{25926886,10263270,4098530,40765625,16591278}},"employment":[{"organization-name":"physcane","start-date":date("2003-03-22")}]}
+{"id":11857618,"id-copy":11857618,"alias":"Glenda","name":"GlendaPyle","user-since":datetime("2009-01-05T13:34:53"),"user-since-copy":datetime("2009-01-05T13:34:53"),"friend-ids":{{31083833,39371819,38336556,7590988,17022330,8016611,41444367,13194826,1589028,37076285,33481940,22093098,9959371,35262849,20744580,33226729,35025566,46396680,30247311,6884899,35691024,40965552,46106170}},"employment":[{"organization-name":"sonstreet","start-date":date("2000-02-19")}]}
+{"id":9985393,"id-copy":9985393,"alias":"Whitaker","name":"WhitakerMang","user-since":datetime("2007-11-28T09:34:34"),"user-since-copy":datetime("2007-11-28T09:34:34"),"friend-ids":{{24107735,37165967,31305236,35313360,9261860,32724193,34416346,8143882,9029425,26723829,4545824}},"employment":[{"organization-name":"Latsonity","start-date":date("2000-08-23"),"end-date":date("2008-08-06")}]}
+{"id":11270020,"id-copy":11270020,"alias":"Ursula","name":"UrsulaSauter","user-since":datetime("2006-09-17T06:18:31"),"user-since-copy":datetime("2006-09-17T06:18:31"),"friend-ids":{{13370394,5537385,6651824,27208272,3304500,26518061,44906267,27803333,8618582,22074752,20865682,15343007}},"employment":[{"organization-name":"Scotcity","start-date":date("2006-08-01")}]}
+{"id":10883062,"id-copy":10883062,"alias":"Lamar","name":"LamarFelbrigge","user-since":datetime("2005-02-12T03:19:28"),"user-since-copy":datetime("2005-02-12T03:19:28"),"friend-ids":{{26304238,21048260,26614197,41153844,17163890,27772117,26679939,22001103,46907785,21321841,46215643,31285577,14997749,46997910,44367495,13858871,20405288,36784906,33752927,30769058,43188289,34006518,23022696}},"employment":[{"organization-name":"Trustbam","start-date":date("2012-06-16")}]}
+{"id":10800157,"id-copy":10800157,"alias":"Tiara","name":"TiaraFuhrer","user-since":datetime("2010-05-24T21:52:36"),"user-since-copy":datetime("2010-05-24T21:52:36"),"friend-ids":{{34031723}},"employment":[{"organization-name":"Sublamdox","start-date":date("2003-03-18"),"end-date":date("2005-09-20")}]}
+{"id":11862502,"id-copy":11862502,"alias":"Innocent","name":"InnocentWilliamson","user-since":datetime("2005-06-09T18:44:51"),"user-since-copy":datetime("2005-06-09T18:44:51"),"friend-ids":{{14750408,36287814,21197416,34246775,18776860,32777856,46956112,18578056,13053407,3282278,29812571,25299530,47168979,6027296,10540009}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2010-05-20"),"end-date":date("2010-01-24")}]}
+{"id":9543280,"id-copy":9543280,"alias":"Isabell","name":"IsabellGaskins","user-since":datetime("2009-12-05T01:29:24"),"user-since-copy":datetime("2009-12-05T01:29:24"),"friend-ids":{{9815607,43778761,25835208,40078303,28971077,9802833,17822058,12655680,37398606,11387722,5483134,11506312,36341116,13511812,3504784,11655484,18350098,15365006,32814750}},"employment":[{"organization-name":"Mathtech","start-date":date("2006-07-01"),"end-date":date("2007-08-14")}]}
+{"id":10678567,"id-copy":10678567,"alias":"Detta","name":"DettaIronmonger","user-since":datetime("2006-05-01T08:52:26"),"user-since-copy":datetime("2006-05-01T08:52:26"),"friend-ids":{{11098679,15763619,12715761,10175990,43581466,4595173,17163835,44918467,38256765,13239047,25476309,9075112,19581524,46478013,24168854,34121818,25604978,21114089}},"employment":[{"organization-name":"Keytech","start-date":date("2007-06-08")}]}
+{"id":9262768,"id-copy":9262768,"alias":"Graham","name":"GrahamHunt","user-since":datetime("2009-03-19T13:15:02"),"user-since-copy":datetime("2009-03-19T13:15:02"),"friend-ids":{{}},"employment":[{"organization-name":"Vivaace","start-date":date("2012-04-23"),"end-date":date("2012-04-15")}]}
+{"id":9896473,"id-copy":9896473,"alias":"Harlan","name":"HarlanAnderson","user-since":datetime("2012-06-03T22:40:33"),"user-since-copy":datetime("2012-06-03T22:40:33"),"friend-ids":{{28073049,32365932,23795268,7563960,47274822,4907078,8659018,33480175,3984139,20631025,26879093,27168884,20063035,22192716,18259756,28904415,28492528,4140983,12014021,10959797,38881978,45835171,6556552,26372018}},"employment":[{"organization-name":"Medflex","start-date":date("2000-08-18")}]}
+{"id":11253043,"id-copy":11253043,"alias":"Joye","name":"JoyeGadow","user-since":datetime("2005-10-03T17:22:30"),"user-since-copy":datetime("2005-10-03T17:22:30"),"friend-ids":{{24978234,7896483,14560795,18402417,16619973,5852675,29679362,19344221,33721635,14137068,30581619,9715250,10966922,24167091,36509340}},"employment":[{"organization-name":"Groovetex","start-date":date("2011-01-08"),"end-date":date("2011-08-10")}]}
+{"id":11427025,"id-copy":11427025,"alias":"Kyran","name":"KyranKlockman","user-since":datetime("2007-11-24T11:35:40"),"user-since-copy":datetime("2007-11-24T11:35:40"),"friend-ids":{{}},"employment":[{"organization-name":"Ontotanin","start-date":date("2004-06-10"),"end-date":date("2008-10-25")}]}
+{"id":10059343,"id-copy":10059343,"alias":"Randy","name":"RandyQueer","user-since":datetime("2005-06-01T02:30:35"),"user-since-copy":datetime("2005-06-01T02:30:35"),"friend-ids":{{8688755,7077909,41009273,26932559,29488059,6408736,6374592,5042147,21880854,12704496,28046022,2384964,20867794,3990470,7132171}},"employment":[{"organization-name":"Dancode","start-date":date("2006-07-07"),"end-date":date("2007-04-08")}]}
+{"id":10505419,"id-copy":10505419,"alias":"Anderson","name":"AndersonSoames","user-since":datetime("2009-04-01T01:24:07"),"user-since-copy":datetime("2009-04-01T01:24:07"),"friend-ids":{{25420744,34012676,8558565,45471514,12117008,35275,4952379,46480100,29394067,15504329,18153717,8476606,19867236,35743164,38523474,6479207,31151752,19687338,5379846,32574974,26920356}},"employment":[{"organization-name":"Doncare","start-date":date("2005-08-01")}]}
+{"id":9695773,"id-copy":9695773,"alias":"Daron","name":"DaronFiddler","user-since":datetime("2006-12-25T17:08:50"),"user-since-copy":datetime("2006-12-25T17:08:50"),"friend-ids":{{14397778,33469556,41690231,7827360,42196316}},"employment":[{"organization-name":"Solfix","start-date":date("2007-12-24")}]}
+{"id":10453837,"id-copy":10453837,"alias":"Leila","name":"LeilaHunter","user-since":datetime("2007-12-08T12:41:34"),"user-since-copy":datetime("2007-12-08T12:41:34"),"friend-ids":{{2310862,19014920}},"employment":[{"organization-name":"Quoline","start-date":date("2011-02-06")}]}
+{"id":9004354,"id-copy":9004354,"alias":"Deshawn","name":"DeshawnGarneys","user-since":datetime("2010-07-21T12:45:03"),"user-since-copy":datetime("2010-07-21T12:45:03"),"friend-ids":{{46096495,1526403}},"employment":[{"organization-name":"Scotcity","start-date":date("2011-07-08")}]}
+{"id":11195221,"id-copy":11195221,"alias":"Clement","name":"ClementBriner","user-since":datetime("2006-12-27T02:29:02"),"user-since-copy":datetime("2006-12-27T02:29:02"),"friend-ids":{{33023290}},"employment":[{"organization-name":"Fixdintex","start-date":date("2000-06-05")}]}
+{"id":9597526,"id-copy":9597526,"alias":"Emory","name":"EmoryThorley","user-since":datetime("2006-01-19T22:44:03"),"user-since-copy":datetime("2006-01-19T22:44:03"),"friend-ids":{{420066,8047878,20510786,1639671,22923859,27319995,3624690,18526424,45857863,2830065,4588990,25531572,17878497,47796172,41309806,34307425,10084701,1659934,38218970,44720636,43501970,610796,35455526,2080900}},"employment":[{"organization-name":"Trustbam","start-date":date("2011-06-18"),"end-date":date("2011-09-10")}]}
+{"id":10703185,"id-copy":10703185,"alias":"Sabina","name":"SabinaHall","user-since":datetime("2012-05-18T20:37:33"),"user-since-copy":datetime("2012-05-18T20:37:33"),"friend-ids":{{432154,6472603,35649237,46598578,35486135,44354453}},"employment":[{"organization-name":"Labzatron","start-date":date("2002-11-04"),"end-date":date("2011-10-12")}]}
+{"id":9226960,"id-copy":9226960,"alias":"Irish","name":"IrishJohnson","user-since":datetime("2009-09-07T21:02:01"),"user-since-copy":datetime("2009-09-07T21:02:01"),"friend-ids":{{4920892,15681759,19110917,26620361,34712468,40890326,20312413}},"employment":[{"organization-name":"over-it","start-date":date("2009-11-11")}]}
+{"id":11536078,"id-copy":11536078,"alias":"Scot","name":"ScotSwartzbaugh","user-since":datetime("2007-06-02T13:28:19"),"user-since-copy":datetime("2007-06-02T13:28:19"),"friend-ids":{{160897,11035428,35908585,14713740,16036400,21530456,31659920,33439685,42771513,42899492,42315848,17885118,12371932,47219421,45350312,33755309,30284897,34557464,21531204,26093690}},"employment":[{"organization-name":"Techitechi","start-date":date("2002-02-23"),"end-date":date("2005-03-24")}]}
+{"id":9945208,"id-copy":9945208,"alias":"Thelma","name":"ThelmaGettemy","user-since":datetime("2006-12-21T11:17:06"),"user-since-copy":datetime("2006-12-21T11:17:06"),"friend-ids":{{26578648,43730418,18099472,11787057,41534206,16778979,41142786,25761045,18556835,25378849,38984390,37528215,2531696}},"employment":[{"organization-name":"tresline","start-date":date("2008-03-25")}]}
+{"id":9288154,"id-copy":9288154,"alias":"Lauren","name":"LaurenGraff","user-since":datetime("2005-12-28T07:21:17"),"user-since-copy":datetime("2005-12-28T07:21:17"),"friend-ids":{{38658043,4029859,43671010,20184796,23429992,3744331,39377881,1336305,33712064,36443}},"employment":[{"organization-name":"strongex","start-date":date("2009-04-06")}]}
+{"id":10714447,"id-copy":10714447,"alias":"Leone","name":"LeoneCoughenour","user-since":datetime("2012-06-13T05:05:11"),"user-since-copy":datetime("2012-06-13T05:05:11"),"friend-ids":{{13098839,21185838,26566436,37464340,8086775,37143068,40377316,39371296}},"employment":[{"organization-name":"Y-geohex","start-date":date("2005-04-16")}]}
+{"id":9318094,"id-copy":9318094,"alias":"Carlo","name":"CarloKelley","user-since":datetime("2012-07-19T09:18:41"),"user-since-copy":datetime("2012-07-19T09:18:41"),"friend-ids":{{39873731,29304807,519851,16423529,10838418,9915172,3040071,39730361,23320290,20572900,7293676,35037765,1744053,38875858}},"employment":[{"organization-name":"Voltlane","start-date":date("2008-08-15")}]}
+{"id":11162920,"id-copy":11162920,"alias":"Michael","name":"MichaelJohns","user-since":datetime("2007-12-21T06:52:31"),"user-since-copy":datetime("2007-12-21T06:52:31"),"friend-ids":{{47587192,5639113,24042062,26141562,4128346,25702038,16421361,44444678,30940270,16928219,27816662,37884076,40854508,21061894,42850960,42453718,2763269,16035171,47650572,26811622}},"employment":[{"organization-name":"Tanzumbam","start-date":date("2003-02-24")}]}
+{"id":11321269,"id-copy":11321269,"alias":"Wilford","name":"WilfordFuhrer","user-since":datetime("2012-01-25T14:53:32"),"user-since-copy":datetime("2012-01-25T14:53:32"),"friend-ids":{{6210425,27216911,3113058,28094966,119775,805604,43386400,46812881,22339620,46498863,26422270,43219229,40022359,39446155}},"employment":[{"organization-name":"Xx-technology","start-date":date("2001-07-06")}]}
+{"id":11542174,"id-copy":11542174,"alias":"Pacey","name":"PaceyTripp","user-since":datetime("2011-11-07T08:36:12"),"user-since-copy":datetime("2011-11-07T08:36:12"),"friend-ids":{{35602078,32622628,34826581,34837077,41522736,14908313,42986568}},"employment":[{"organization-name":"goldendexon","start-date":date("2006-07-08")}]}
+{"id":11638618,"id-copy":11638618,"alias":"Garfield","name":"GarfieldHardie","user-since":datetime("2007-07-05T04:44:27"),"user-since-copy":datetime("2007-07-05T04:44:27"),"friend-ids":{{47307628,3109848,30936899,7173119,33551634,24239136,11619168,633835,34791947,12052833,19798108,3426648,395456,18555868,18509839,8340275,14943912,42330581,313099,25632353,27912788,20281899,8961605,13625222}},"employment":[{"organization-name":"Technohow","start-date":date("2001-02-24")}]}
+{"id":10902049,"id-copy":10902049,"alias":"Fae","name":"FaeRing","user-since":datetime("2008-06-15T12:54:57"),"user-since-copy":datetime("2008-06-15T12:54:57"),"friend-ids":{{2667467,46445373,11696423,42003744,47667382,34088774,4279683,29934858,21213543,44195034,38786294,14946433,38805114,9972575,3309290,5324029,32663319,20577589,9110909,27272396,47622938}},"employment":[{"organization-name":"Dandamace","start-date":date("2007-11-15")}]}
+{"id":11735830,"id-copy":11735830,"alias":"Maryvonne","name":"MaryvonneHarrold","user-since":datetime("2007-12-03T06:30:43"),"user-since-copy":datetime("2007-12-03T06:30:43"),"friend-ids":{{27842540,46624942,21701969,33750891,28523702,38840881,1497785,32357938,19740312,1880841,41116687,35621654,46917268,14610853,33099367,8710534}},"employment":[{"organization-name":"Indiex","start-date":date("2004-10-04")}]}
+{"id":9760834,"id-copy":9760834,"alias":"Lavette","name":"LavettePirl","user-since":datetime("2006-02-12T07:28:53"),"user-since-copy":datetime("2006-02-12T07:28:53"),"friend-ids":{{27450797,36415787}},"employment":[{"organization-name":"Kanelectrics","start-date":date("2002-09-20")}]}
+{"id":11996683,"id-copy":11996683,"alias":"Ivy","name":"IvyReddish","user-since":datetime("2008-10-09T09:54:46"),"user-since-copy":datetime("2008-10-09T09:54:46"),"friend-ids":{{42344158,40312093,15782003}},"employment":[{"organization-name":"Hot-tech","start-date":date("2003-04-16")}]}
+{"id":9574261,"id-copy":9574261,"alias":"Kalysta","name":"KalystaBeedell","user-since":datetime("2010-01-27T14:57:31"),"user-since-copy":datetime("2010-01-27T14:57:31"),"friend-ids":{{5811189,22155580,41736564,27399656,40013573,28340467,45690668,16097604,9655169,44870593}},"employment":[{"organization-name":"Tripplelane","start-date":date("2009-12-16"),"end-date":date("2010-10-22")}]}
+{"id":9450532,"id-copy":9450532,"alias":"Troy","name":"TroyKoepple","user-since":datetime("2011-05-10T09:56:46"),"user-since-copy":datetime("2011-05-10T09:56:46"),"friend-ids":{{42029412,18025243,715282,501115,38550360,39016114,31451417,38836992,13665836,17286159,28850827,17241066,41893804,39172781,4523003,28542863,25386847,44039032,19593806,607220,26442265,47847281}},"employment":[{"organization-name":"Dandamace","start-date":date("2002-12-26"),"end-date":date("2004-04-05")}]}
+{"id":11111890,"id-copy":11111890,"alias":"Geordie","name":"GeordieGraff","user-since":datetime("2006-02-12T04:30:44"),"user-since-copy":datetime("2006-02-12T04:30:44"),"friend-ids":{{12852237,10391003,37679153,6620205,25381043,19805548,4534765,11626709,47369482,15045527,25177819,15113002,39634176,40637870,47662386,8045236}},"employment":[{"organization-name":"linedexon","start-date":date("2005-06-18")}]}
+{"id":10937395,"id-copy":10937395,"alias":"Madlyn","name":"MadlynRader","user-since":datetime("2010-11-11T02:19:12"),"user-since-copy":datetime("2010-11-11T02:19:12"),"friend-ids":{{8750346,40237703,11127018,23810876,33862918,8179642}},"employment":[{"organization-name":"Sanjodax","start-date":date("2011-03-12"),"end-date":date("2011-12-06")}]}
+{"id":9502096,"id-copy":9502096,"alias":"Hebe","name":"HebeEndsley","user-since":datetime("2012-08-08T18:55:28"),"user-since-copy":datetime("2012-08-08T18:55:28"),"friend-ids":{{34917916,5530270,12994124,25113086,28819142,44228082}},"employment":[{"organization-name":"Newcom","start-date":date("2007-04-11")}]}
+{"id":11456404,"id-copy":11456404,"alias":"Lonny","name":"LonnyUllman","user-since":datetime("2008-10-19T03:05:07"),"user-since-copy":datetime("2008-10-19T03:05:07"),"friend-ids":{{30675414,44654756,8273748,12998719,20082930}},"employment":[{"organization-name":"over-it","start-date":date("2009-11-02"),"end-date":date("2011-05-11")}]}
+{"id":10582339,"id-copy":10582339,"alias":"Randall","name":"RandallDrabble","user-since":datetime("2006-09-08T10:08:58"),"user-since-copy":datetime("2006-09-08T10:08:58"),"friend-ids":{{32686522,24466673,14026712,31573032,14639819,19975138,30208386,24174917,7234882,9431452,18256175,18934583,31539286,46107937,32747992,28900739,40079932,40674667,33527888,45927633,22350243,14260823,19696930,17970296}},"employment":[{"organization-name":"Ganjastrip","start-date":date("2008-12-13")}]}
+{"id":11072782,"id-copy":11072782,"alias":"Jewel","name":"JewelSchreckengost","user-since":datetime("2012-06-04T18:20:29"),"user-since-copy":datetime("2012-06-04T18:20:29"),"friend-ids":{{47896348,34649239,38135221,19731900,14383059,3639686,28133949,1326525,415048,34486382,32809579,31754806,33563370}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2008-03-06")}]}
+{"id":10655089,"id-copy":10655089,"alias":"Quinn","name":"QuinnHays","user-since":datetime("2009-11-25T04:42:39"),"user-since-copy":datetime("2009-11-25T04:42:39"),"friend-ids":{{17385636,24378500,37614592,32315940,18046144,45823175,29709981,28423306,23783823,10623867,27782698}},"employment":[{"organization-name":"Tanzumbam","start-date":date("2012-06-09")}]}
+{"id":11289733,"id-copy":11289733,"alias":"Jettie","name":"JettieElinor","user-since":datetime("2006-03-02T09:44:17"),"user-since-copy":datetime("2006-03-02T09:44:17"),"friend-ids":{{}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2002-07-25"),"end-date":date("2005-01-16")}]}
+{"id":9952342,"id-copy":9952342,"alias":"Christal","name":"ChristalMcmichaels","user-since":datetime("2008-02-13T13:25:45"),"user-since-copy":datetime("2008-02-13T13:25:45"),"friend-ids":{{12290348,1563117,10883525,17285406,3798829,3734533,13084348,31001579,23655942,44480002,11803789,8240833,42718608,41919526,37582304,10494964,10815416,10676699,9376307}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2011-05-16")}]}
+{"id":11136910,"id-copy":11136910,"alias":"Karl","name":"KarlGarratt","user-since":datetime("2006-12-22T01:58:50"),"user-since-copy":datetime("2006-12-22T01:58:50"),"friend-ids":{{753124,31382435,30698735,25951267,27027532,34551403,9451765,37517863,3719825,37613952,18670991,39783690,6592095,27477830,31739951,24458195,12317249}},"employment":[{"organization-name":"Ontohothex","start-date":date("2011-05-11")}]}
+{"id":9430849,"id-copy":9430849,"alias":"Emil","name":"EmilGarland","user-since":datetime("2008-07-03T15:56:07"),"user-since-copy":datetime("2008-07-03T15:56:07"),"friend-ids":{{40429008,45432330,22293451,2129366,19514477,20108162,28656704,35403173,33855801,14660181}},"employment":[{"organization-name":"Xx-drill","start-date":date("2010-02-10")}]}
+{"id":9317395,"id-copy":9317395,"alias":"Timothy","name":"TimothyMays","user-since":datetime("2007-05-23T15:42:26"),"user-since-copy":datetime("2007-05-23T15:42:26"),"friend-ids":{{38066468,16126194,20685050,8542551,36810930,36333903,31522960,44908120,45171970,9212095,16986466,41689196,22300874,45983009,30918582,5896299,2682406,6649020,33199300,14523848}},"employment":[{"organization-name":"Doncare","start-date":date("2006-04-16"),"end-date":date("2008-02-21")}]}
+{"id":11049715,"id-copy":11049715,"alias":"Carlo","name":"CarloBrooks","user-since":datetime("2005-03-23T21:46:06"),"user-since-copy":datetime("2005-03-23T21:46:06"),"friend-ids":{{8214850,7465603,15385071,32299168,5993026,3262895,24995417,25987462,10230501,12537459,44597291,33492282,30758369,15589085,6799067,23023304,42597416,10978280,40668626,25650335,37336071}},"employment":[{"organization-name":"jaydax","start-date":date("2011-09-15"),"end-date":date("2011-09-03")}]}
+{"id":9389254,"id-copy":9389254,"alias":"Jon","name":"JonShaw","user-since":datetime("2006-12-10T11:28:23"),"user-since-copy":datetime("2006-12-10T11:28:23"),"friend-ids":{{}},"employment":[{"organization-name":"Latsonity","start-date":date("2010-07-24")}]}
+{"id":9379975,"id-copy":9379975,"alias":"Kyra","name":"KyraLangston","user-since":datetime("2012-01-18T06:06:56"),"user-since-copy":datetime("2012-01-18T06:06:56"),"friend-ids":{{46662872,1388016,21715152,3266023,18080709,25857347,29710885,22300787,25086634,25220921,17189604,21754574,27820275,7441940,10911235,46304871,6518794}},"employment":[{"organization-name":"Kongreen","start-date":date("2008-04-03"),"end-date":date("2008-04-07")}]}
+{"id":11529364,"id-copy":11529364,"alias":"Rufus","name":"RufusGreen","user-since":datetime("2009-04-14T15:51:24"),"user-since-copy":datetime("2009-04-14T15:51:24"),"friend-ids":{{5011595}},"employment":[{"organization-name":"Scotcity","start-date":date("2000-09-25"),"end-date":date("2004-08-22")}]}
+{"id":11039716,"id-copy":11039716,"alias":"Piedad","name":"PiedadHowe","user-since":datetime("2011-02-23T17:18:37"),"user-since-copy":datetime("2011-02-23T17:18:37"),"friend-ids":{{13323345}},"employment":[{"organization-name":"Keytech","start-date":date("2009-03-26"),"end-date":date("2009-06-17")}]}
+{"id":10212385,"id-copy":10212385,"alias":"Alice","name":"AliceJones","user-since":datetime("2009-05-16T16:08:03"),"user-since-copy":datetime("2009-05-16T16:08:03"),"friend-ids":{{4158604,3204211,21491737,39619715,9750334}},"employment":[{"organization-name":"Viatechi","start-date":date("2012-04-19")}]}
+{"id":9845113,"id-copy":9845113,"alias":"Chia","name":"ChiaGeddinge","user-since":datetime("2008-12-12T16:50:57"),"user-since-copy":datetime("2008-12-12T16:50:57"),"friend-ids":{{16725476,120161,762756,40795640,34195102,27938737}},"employment":[{"organization-name":"Sublamdox","start-date":date("2001-01-03"),"end-date":date("2001-11-03")}]}
+{"id":11445889,"id-copy":11445889,"alias":"Milford","name":"MilfordTeagarden","user-since":datetime("2006-06-07T19:18:28"),"user-since-copy":datetime("2006-06-07T19:18:28"),"friend-ids":{{}},"employment":[{"organization-name":"subtam","start-date":date("2003-07-26")}]}
+{"id":11244439,"id-copy":11244439,"alias":"Francene","name":"FranceneArmstrong","user-since":datetime("2009-11-12T19:32:27"),"user-since-copy":datetime("2009-11-12T19:32:27"),"friend-ids":{{27784445,37528954,14014093,18695376}},"employment":[{"organization-name":"Xx-drill","start-date":date("2000-06-26")}]}
+{"id":10494370,"id-copy":10494370,"alias":"Maria","name":"MariaToke","user-since":datetime("2009-12-06T17:40:38"),"user-since-copy":datetime("2009-12-06T17:40:38"),"friend-ids":{{28240347,34042532}},"employment":[{"organization-name":"Streettax","start-date":date("2001-08-08"),"end-date":date("2008-07-09")}]}
+{"id":10868761,"id-copy":10868761,"alias":"Peronel","name":"PeronelGongaware","user-since":datetime("2010-01-25T14:26:30"),"user-since-copy":datetime("2010-01-25T14:26:30"),"friend-ids":{{28271989,41567995,31926358,16420360,15775849,44023747,39099521,4517209,39890594,39784644,43247769,25427216,46426794,37704581,46477208,3213706}},"employment":[{"organization-name":"Coneflex","start-date":date("2011-12-16")}]}
+{"id":9970132,"id-copy":9970132,"alias":"Garrett","name":"GarrettPery","user-since":datetime("2007-03-03T11:19:29"),"user-since-copy":datetime("2007-03-03T11:19:29"),"friend-ids":{{25744707,31991833,37406793,30461766,24815522,3640470,13669903,17663561,19222132,29107132,42516393,40032051,24029037,28047983,45579233}},"employment":[{"organization-name":"Sanjodax","start-date":date("2001-01-12")}]}
+{"id":10258114,"id-copy":10258114,"alias":"Chuck","name":"ChuckGibson","user-since":datetime("2012-07-20T03:48:15"),"user-since-copy":datetime("2012-07-20T03:48:15"),"friend-ids":{{32318205,37049120,26298456,3281723,14892306,29998569,29992020,36383932,15333422,29670243}},"employment":[{"organization-name":"Medflex","start-date":date("2006-10-11"),"end-date":date("2011-09-02")}]}
+{"id":10349656,"id-copy":10349656,"alias":"Woodrow","name":"WoodrowRichter","user-since":datetime("2006-09-18T16:22:12"),"user-since-copy":datetime("2006-09-18T16:22:12"),"friend-ids":{{12344306,36484394,30889842,47572749,42102868,22350773,7166034,16132372,45197714,34516830,47108654,4399888,24401048,32578065,16593311,33394001,7356357,29943304,30866764,11942891}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2003-11-18"),"end-date":date("2004-10-16")}]}
+{"id":11415055,"id-copy":11415055,"alias":"Zavia","name":"ZaviaLombardi","user-since":datetime("2006-01-10T02:11:24"),"user-since-copy":datetime("2006-01-10T02:11:24"),"friend-ids":{{25953753,952678,31067065}},"employment":[{"organization-name":"Ganjatax","start-date":date("2003-06-27"),"end-date":date("2010-07-02")}]}
+{"id":9221836,"id-copy":9221836,"alias":"Claud","name":"ClaudPratt","user-since":datetime("2008-01-01T04:10:02"),"user-since-copy":datetime("2008-01-01T04:10:02"),"friend-ids":{{35586361,40548794,7169299,24675214,21079165,37323851,16881366,24433012,38047831,34495409,33711705,8957126,38345318}},"employment":[{"organization-name":"Newphase","start-date":date("2002-12-05")}]}
+{"id":11587057,"id-copy":11587057,"alias":"Meagan","name":"MeaganHays","user-since":datetime("2012-08-15T21:45:05"),"user-since-copy":datetime("2012-08-15T21:45:05"),"friend-ids":{{26887765,1940688,10308941,42037682,1716669,38995955,17690888,23227010,4054166,22275630,6863237,15140164,38703696,19044355,43996569,12255978,28516070}},"employment":[{"organization-name":"Doncare","start-date":date("2003-02-26"),"end-date":date("2010-08-05")}]}
+{"id":9512989,"id-copy":9512989,"alias":"Lilliana","name":"LillianaAdams","user-since":datetime("2007-06-01T16:54:29"),"user-since-copy":datetime("2007-06-01T16:54:29"),"friend-ids":{{14085316,47471900,24950195,44416851,6677091,34188319,1783776,35860593,29193624,11999697,13365419,39452732,14401842,9087264,15679216,39424118,45063958,11967959,29634503,15763396}},"employment":[{"organization-name":"Fixdintex","start-date":date("2001-02-08"),"end-date":date("2008-03-23")}]}
+{"id":11306677,"id-copy":11306677,"alias":"Chong","name":"ChongPawle","user-since":datetime("2007-09-13T00:31:41"),"user-since-copy":datetime("2007-09-13T00:31:41"),"friend-ids":{{11341417,23669364,41504484,29889550,268223,26888454,43915376,23795433,14021648,25630355,19831181,15828987}},"employment":[{"organization-name":"Fax-fax","start-date":date("2011-01-06"),"end-date":date("2011-10-06")}]}
+{"id":11782354,"id-copy":11782354,"alias":"Glynda","name":"GlyndaEnderly","user-since":datetime("2007-11-25T06:01:45"),"user-since-copy":datetime("2007-11-25T06:01:45"),"friend-ids":{{16202981,24035766,10175614,27353200,26183740,6084065,31664832,22446721,2792685,37521374,1999182,12494503,18087992,44433851}},"employment":[{"organization-name":"Y-geohex","start-date":date("2004-06-10")}]}
+{"id":11007700,"id-copy":11007700,"alias":"Elly","name":"EllyWard","user-since":datetime("2009-04-20T08:46:09"),"user-since-copy":datetime("2009-04-20T08:46:09"),"friend-ids":{{9712756,6523354}},"employment":[{"organization-name":"Icerunin","start-date":date("2007-12-07"),"end-date":date("2007-07-27")}]}
+{"id":10970950,"id-copy":10970950,"alias":"Shana","name":"ShanaRose","user-since":datetime("2008-09-17T10:03:01"),"user-since-copy":datetime("2008-09-17T10:03:01"),"friend-ids":{{21025589,17977659,39920039,44311386,2634251}},"employment":[{"organization-name":"Sanjodax","start-date":date("2006-09-17")}]}
+{"id":9640915,"id-copy":9640915,"alias":"Harrison","name":"HarrisonHildyard","user-since":datetime("2009-05-25T11:56:05"),"user-since-copy":datetime("2009-05-25T11:56:05"),"friend-ids":{{41488832,16139664,18327029,38811764,38271538,13106137,26450611,11574808,33108523,31639017,9208159,18456510,47955463,2606160,29293146,13981743,39967993,23629640,32666499,35046044,2402842,1117025,17741007,14997808}},"employment":[{"organization-name":"Hatcom","start-date":date("2010-03-06")}]}
+{"id":11888530,"id-copy":11888530,"alias":"Louis","name":"LouisRichards","user-since":datetime("2011-10-26T02:27:49"),"user-since-copy":datetime("2011-10-26T02:27:49"),"friend-ids":{{40512993,46289399}},"employment":[{"organization-name":"Qvohouse","start-date":date("2000-04-18"),"end-date":date("2002-08-03")}]}
+{"id":9199078,"id-copy":9199078,"alias":"Erwin","name":"ErwinErrett","user-since":datetime("2011-04-20T12:44:31"),"user-since-copy":datetime("2011-04-20T12:44:31"),"friend-ids":{{31928109,8101864,44247743,21370948}},"employment":[{"organization-name":"Medflex","start-date":date("2000-03-06")}]}
+{"id":10943104,"id-copy":10943104,"alias":"Prudence","name":"PrudencePriebe","user-since":datetime("2006-04-27T21:00:43"),"user-since-copy":datetime("2006-04-27T21:00:43"),"friend-ids":{{43633941,38710166,34456560,11324015,21000755,23356715,21056830,27295754}},"employment":[{"organization-name":"Ontotanin","start-date":date("2012-08-30")}]}
+{"id":10185346,"id-copy":10185346,"alias":"Noah","name":"NoahAshmore","user-since":datetime("2006-04-04T14:33:43"),"user-since-copy":datetime("2006-04-04T14:33:43"),"friend-ids":{{15819384,46052301,7102428,7977240,30337629,31480307,30013142,4192580,34814572,6841517,2253788,31150059,505825,27897490,11402219}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2012-06-06")}]}
+{"id":9680644,"id-copy":9680644,"alias":"Mirtha","name":"MirthaRahl","user-since":datetime("2008-02-09T04:05:03"),"user-since-copy":datetime("2008-02-09T04:05:03"),"friend-ids":{{25328638,9009324,16627989,46602908,32685062,10538437,22403363,4205292,27910567,28430833,8519372,39774027,12120028,1211979}},"employment":[{"organization-name":"Lexitechno","start-date":date("2006-12-19")}]}
+{"id":10905721,"id-copy":10905721,"alias":"Tibby","name":"TibbyPriebe","user-since":datetime("2010-04-09T18:32:02"),"user-since-copy":datetime("2010-04-09T18:32:02"),"friend-ids":{{18406663,1072532,16897765}},"employment":[{"organization-name":"Solophase","start-date":date("2007-05-05"),"end-date":date("2007-03-06")}]}
+{"id":11066710,"id-copy":11066710,"alias":"Caryl","name":"CarylMaugham","user-since":datetime("2007-02-10T03:38:03"),"user-since-copy":datetime("2007-02-10T03:38:03"),"friend-ids":{{41776362,7370825,35851510,23733011,27617379,39377372,3043067,22122576,11996852,20708849,40772627,20108470,4141780,3724555,31849764,7347633}},"employment":[{"organization-name":"Villa-tech","start-date":date("2001-10-15")}]}
+{"id":11471689,"id-copy":11471689,"alias":"Bevis","name":"BevisWhishaw","user-since":datetime("2011-03-05T23:14:53"),"user-since-copy":datetime("2011-03-05T23:14:53"),"friend-ids":{{27818002,43784015,39101258,28170566,38541659,43935487,907437,25457112,4731176,35304801,30364855,33197014,27028915,21746182,47624076,41599425,8592245}},"employment":[{"organization-name":"geomedia","start-date":date("2000-04-04"),"end-date":date("2009-05-08")}]}
+{"id":11937787,"id-copy":11937787,"alias":"Addison","name":"AddisonEckert","user-since":datetime("2007-04-26T01:06:38"),"user-since-copy":datetime("2007-04-26T01:06:38"),"friend-ids":{{6446414,23134374,38952228,25368200,47868440,29231397,15672064,2482344,22824732,13563448,43826877}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2009-10-09")}]}
+{"id":9174313,"id-copy":9174313,"alias":"Hal","name":"HalHasely","user-since":datetime("2008-01-28T17:01:16"),"user-since-copy":datetime("2008-01-28T17:01:16"),"friend-ids":{{9058102,40616538,45706325,991699,37832260,4793008,36372035,23272432,36685642,2621984,9576806,14325601,41449409,16499609,20610820,1564035,20738111,19735088,31942764,34813086}},"employment":[{"organization-name":"Mathtech","start-date":date("2006-10-16")}]}
+{"id":10138039,"id-copy":10138039,"alias":"Farah","name":"FarahAnn","user-since":datetime("2008-05-10T19:04:28"),"user-since-copy":datetime("2008-05-10T19:04:28"),"friend-ids":{{32501277,13715476,10452566,2652600,16449577,12508457,30925424,21595197,26030962,31683678}},"employment":[{"organization-name":"tresline","start-date":date("2003-10-02")}]}
+{"id":10955896,"id-copy":10955896,"alias":"Felton","name":"FeltonRiggle","user-since":datetime("2010-08-18T08:55:19"),"user-since-copy":datetime("2010-08-18T08:55:19"),"friend-ids":{{9250996,46302470,16921353,21053478,40274566,25492381,7743899}},"employment":[{"organization-name":"Quadlane","start-date":date("2008-09-10"),"end-date":date("2009-01-22")}]}
+{"id":10136659,"id-copy":10136659,"alias":"Robt","name":"RobtKooser","user-since":datetime("2008-11-08T19:22:49"),"user-since-copy":datetime("2008-11-08T19:22:49"),"friend-ids":{{22245145,29285750,9880896}},"employment":[{"organization-name":"Coneflex","start-date":date("2001-02-07")}]}
+{"id":11542519,"id-copy":11542519,"alias":"Colten","name":"ColtenDemuth","user-since":datetime("2012-02-09T01:22:04"),"user-since-copy":datetime("2012-02-09T01:22:04"),"friend-ids":{{15666280,36489446,45424145,47509110,24198688,42545568,30526545,43828073,26402530,23632737,20385217,35055795,38789042,34967858,521531,47834820,20307524}},"employment":[{"organization-name":"Techitechi","start-date":date("2008-04-10")}]}
+{"id":9471385,"id-copy":9471385,"alias":"Weldon","name":"WeldonMaclagan","user-since":datetime("2010-01-24T22:21:59"),"user-since-copy":datetime("2010-01-24T22:21:59"),"friend-ids":{{42864267,16710494,27436346,7324905,3901396,11812437,31490561,3906397}},"employment":[{"organization-name":"Quadlane","start-date":date("2002-09-07"),"end-date":date("2006-07-08")}]}
+{"id":11370337,"id-copy":11370337,"alias":"Devin","name":"DevinWatson","user-since":datetime("2009-07-19T11:47:07"),"user-since-copy":datetime("2009-07-19T11:47:07"),"friend-ids":{{25117468,31957773,46217915,26169035,34203342,32134285,10572760,10974016,33771064,4177645,4910095,18301833,15264956,5806057,37899843,35459189,4391801,34940818}},"employment":[{"organization-name":"Freshfix","start-date":date("2008-06-19")}]}
+{"id":10001410,"id-copy":10001410,"alias":"Denzil","name":"DenzilLedgerwood","user-since":datetime("2006-12-24T10:56:58"),"user-since-copy":datetime("2006-12-24T10:56:58"),"friend-ids":{{25633920,39748697,3557647,44396047,25225495,38723684,5854330}},"employment":[{"organization-name":"Xx-drill","start-date":date("2000-08-14"),"end-date":date("2011-07-20")}]}
+{"id":9577729,"id-copy":9577729,"alias":"Jann","name":"JannPorter","user-since":datetime("2006-05-03T08:57:08"),"user-since-copy":datetime("2006-05-03T08:57:08"),"friend-ids":{{7711959,4131696,10146353,46418552,37999454,38333059,16381326,45028736,16829150}},"employment":[{"organization-name":"Striptaxon","start-date":date("2006-10-19")}]}
+{"id":9045535,"id-copy":9045535,"alias":"Ebenezer","name":"EbenezerPery","user-since":datetime("2008-06-05T17:48:45"),"user-since-copy":datetime("2008-06-05T17:48:45"),"friend-ids":{{}},"employment":[{"organization-name":"jaydax","start-date":date("2012-04-07"),"end-date":date("2012-06-10")}]}
+{"id":9155080,"id-copy":9155080,"alias":"Errol","name":"ErrolLittle","user-since":datetime("2011-12-20T07:09:25"),"user-since-copy":datetime("2011-12-20T07:09:25"),"friend-ids":{{17400275,40794627,12632163,45365986,7980045,7368579,40357205,29279590,258707,38447445,27048261,19911849,10768265,24278809,11940146,33555290,23286799,40641141,33877442}},"employment":[{"organization-name":"Greencare","start-date":date("2004-03-05")}]}
+{"id":9811513,"id-copy":9811513,"alias":"Casie","name":"CasieRose","user-since":datetime("2011-11-25T11:32:36"),"user-since-copy":datetime("2011-11-25T11:32:36"),"friend-ids":{{8913855,26924028,19426899,38037518,39689117,32691982,6561788,36463261,31724455,18356325,23130893,35227626,13738524,4700460,6963740,13255939,12215189,33593825,34229322}},"employment":[{"organization-name":"jaydax","start-date":date("2003-11-22")}]}
+{"id":10936798,"id-copy":10936798,"alias":"Chang","name":"ChangBriner","user-since":datetime("2011-01-21T02:58:13"),"user-since-copy":datetime("2011-01-21T02:58:13"),"friend-ids":{{44173597,3293094,47813131,8981206,36324479,16594808,20038389,11223092,7224123,10682354,7270314,5170866,10241023,43090387,21910381,36504407,18319458,19534667,14493618,11394344,5990164,35322441}},"employment":[{"organization-name":"Icerunin","start-date":date("2004-12-09"),"end-date":date("2006-08-28")}]}
+{"id":11437771,"id-copy":11437771,"alias":"Brittani","name":"BrittaniMoore","user-since":datetime("2007-11-16T20:56:35"),"user-since-copy":datetime("2007-11-16T20:56:35"),"friend-ids":{{30502334,18483492,37360877,25153720,9181228,28352241,37928337,13522608,20974146,30187156,22832401,20899789,44606652,3333090,39581573,34303132,33802071,27053375,32467186,40213342,37254307,7275338,2622767}},"employment":[{"organization-name":"Quadlane","start-date":date("2010-02-07")}]}
+{"id":10783822,"id-copy":10783822,"alias":"Emerald","name":"EmeraldMillard","user-since":datetime("2008-08-07T16:33:44"),"user-since-copy":datetime("2008-08-07T16:33:44"),"friend-ids":{{22464360,7890894,18256597,33659179,24554534,30962087,29716339,23689397,45113518,19997635}},"employment":[{"organization-name":"Coneflex","start-date":date("2001-06-10"),"end-date":date("2006-12-02")}]}
+{"id":10760020,"id-copy":10760020,"alias":"Emeline","name":"EmelineCowher","user-since":datetime("2006-03-11T07:02:10"),"user-since-copy":datetime("2006-03-11T07:02:10"),"friend-ids":{{2652618,22247716,39487944,16288504,8109009,34390947,2041892,27800644,5979423,12674908}},"employment":[{"organization-name":"sonstreet","start-date":date("2007-12-26"),"end-date":date("2007-09-04")}]}
+{"id":11268778,"id-copy":11268778,"alias":"Chuck","name":"ChuckRamos","user-since":datetime("2005-09-24T12:19:57"),"user-since-copy":datetime("2005-09-24T12:19:57"),"friend-ids":{{2142650,15399676,40659179,32507535,32269323,46947373,46293990,4237301,41447393,21345670,47299716,8515646,27204593,6676856,21757183,13647535,28951520,23198255,1618106,18189425,46835891,7056692,26622607}},"employment":[{"organization-name":"Indiex","start-date":date("2004-06-24"),"end-date":date("2006-01-05")}]}
+{"id":9453925,"id-copy":9453925,"alias":"Ritchie","name":"RitchieJube","user-since":datetime("2008-04-28T12:33:34"),"user-since-copy":datetime("2008-04-28T12:33:34"),"friend-ids":{{44327769,45189889,11098478,41612069,40647950,638474,21614810,22273745,6230791,15120137,18477729,16895919,5907839,43993812,31639138,7966991,11024409}},"employment":[{"organization-name":"Medflex","start-date":date("2012-07-22")}]}
+{"id":9461098,"id-copy":9461098,"alias":"Teodoro","name":"TeodoroBullard","user-since":datetime("2010-07-24T07:40:44"),"user-since-copy":datetime("2010-07-24T07:40:44"),"friend-ids":{{8278091,1756629,9893864,11184021,2292251,20614604,48014557,23491569,11328678,11572435,45790306,44930978,34910222,16655255,29338869,27169036,19669405,20512510,33598988,38104427}},"employment":[{"organization-name":"Greencare","start-date":date("2003-01-17"),"end-date":date("2007-05-28")}]}
+{"id":9396193,"id-copy":9396193,"alias":"Franklyn","name":"FranklynVorrasi","user-since":datetime("2007-06-27T09:38:03"),"user-since-copy":datetime("2007-06-27T09:38:03"),"friend-ids":{{12870114,28811462,19219273,38745339,22310708,11419733,21583164,42276545,1177024,43617748,11702666,19332437,1523883,40265275,41227772}},"employment":[{"organization-name":"over-it","start-date":date("2001-03-13"),"end-date":date("2009-02-07")}]}
+{"id":11788345,"id-copy":11788345,"alias":"Mindy","name":"MindyRockwell","user-since":datetime("2011-02-20T23:55:16"),"user-since-copy":datetime("2011-02-20T23:55:16"),"friend-ids":{{7821092,24614722,27718237,19686343,43916267,7882804,34422272,46273261,658009,42620170,36177155,3340224,27157340,20438623,19694381,15643415,43465380,17719224,37073374,42060457,29532671,3781069,26121650}},"employment":[{"organization-name":"Quoline","start-date":date("2011-05-11")}]}
+{"id":9588427,"id-copy":9588427,"alias":"Tiffany","name":"TiffanyGeyer","user-since":datetime("2007-09-10T11:20:53"),"user-since-copy":datetime("2007-09-10T11:20:53"),"friend-ids":{{31357437,16305152,39281885,25249419,434661,13634747,39812462,25218908,22362649,41696008,4523776,40340358,45330588,299997,11538141,20972409,25152923,8627592,33381524,6226232}},"employment":[{"organization-name":"subtam","start-date":date("2005-02-20")}]}
+{"id":11403742,"id-copy":11403742,"alias":"Neil","name":"NeilHobbs","user-since":datetime("2012-02-26T07:07:17"),"user-since-copy":datetime("2012-02-26T07:07:17"),"friend-ids":{{28387528,39844931,32868894,45540524,35239986,44255870,20859099}},"employment":[{"organization-name":"Fix-touch","start-date":date("2008-11-28"),"end-date":date("2009-06-01")}]}
+{"id":9748939,"id-copy":9748939,"alias":"April","name":"AprilCourtney","user-since":datetime("2008-02-10T17:35:28"),"user-since-copy":datetime("2008-02-10T17:35:28"),"friend-ids":{{43018591,38860193,26524230,23704979,2293321,18201469,41569073,26942967,16348102,20218840,30888146,7584389,11355443,3703344}},"employment":[{"organization-name":"Newphase","start-date":date("2001-08-26")}]}
+{"id":10663741,"id-copy":10663741,"alias":"Gaylord","name":"GaylordWynne","user-since":datetime("2007-09-07T09:15:35"),"user-since-copy":datetime("2007-09-07T09:15:35"),"friend-ids":{{34508923,28228552,7714885,16525247,30914675,8152699,26553788,8070452,45739728}},"employment":[{"organization-name":"Solfix","start-date":date("2006-05-18"),"end-date":date("2008-04-07")}]}
+{"id":9922381,"id-copy":9922381,"alias":"Cecilia","name":"CeciliaOsteen","user-since":datetime("2009-06-03T03:58:36"),"user-since-copy":datetime("2009-06-03T03:58:36"),"friend-ids":{{22246989,9095240,8953245,16326669,38845534,13608449,35076758,42004583}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2010-08-02")}]}
+{"id":10650526,"id-copy":10650526,"alias":"Gertie","name":"GertieWallace","user-since":datetime("2010-07-16T05:33:07"),"user-since-copy":datetime("2010-07-16T05:33:07"),"friend-ids":{{35934417,43053648,35859770,43704932,35605486,17212020,21235775,26783725,17450538,42996452,15873053,36331217,18524993,45483950,1549676,24801562,46527491}},"employment":[{"organization-name":"jaydax","start-date":date("2003-08-16")}]}
+{"id":9343705,"id-copy":9343705,"alias":"Ramsey","name":"RamseyWarner","user-since":datetime("2006-04-24T09:52:39"),"user-since-copy":datetime("2006-04-24T09:52:39"),"friend-ids":{{36909861,36881715,40993685,18669519,42428458,2780280,6070725,10466662,26215221,16329040,38464211,14024902,8083000,27857433,14282674,1976238,6345526,35452338,21503723,34910137,26860195,426384,27759959}},"employment":[{"organization-name":"Greencare","start-date":date("2003-10-28")}]}
+{"id":10733305,"id-copy":10733305,"alias":"Dakota","name":"DakotaSmith","user-since":datetime("2009-11-17T19:52:42"),"user-since-copy":datetime("2009-11-17T19:52:42"),"friend-ids":{{21984282,14492326,18724474,17361116,26773641,32118673,8295454,6804824}},"employment":[{"organization-name":"Newcom","start-date":date("2007-05-28")}]}
+{"id":11616628,"id-copy":11616628,"alias":"Jessamine","name":"JessamineWolff","user-since":datetime("2008-05-03T17:05:35"),"user-since-copy":datetime("2008-05-03T17:05:35"),"friend-ids":{{38285911,42183685,11422759,25927239,22771435,47814309,43146385,39761181,1670925,15764683,8068597,3561105}},"employment":[{"organization-name":"Trustbam","start-date":date("2007-07-26"),"end-date":date("2010-03-16")}]}
+{"id":10162495,"id-copy":10162495,"alias":"Malina","name":"MalinaTrout","user-since":datetime("2006-12-19T12:12:55"),"user-since-copy":datetime("2006-12-19T12:12:55"),"friend-ids":{{40578475,43374248,7059820,18838227,45149295,47680877,11640348,19081155,9959453,46807478,45192583,39333999,4869981,42888726,32789666,19653202}},"employment":[{"organization-name":"Whitemedia","start-date":date("2000-11-08")}]}
+{"id":10357477,"id-copy":10357477,"alias":"Rosy","name":"RosyMitchell","user-since":datetime("2005-08-13T13:44:24"),"user-since-copy":datetime("2005-08-13T13:44:24"),"friend-ids":{{13370964,4479736,44060098,28936173,42239651,18380035,17854869,36485096,7662833}},"employment":[{"organization-name":"kin-ron","start-date":date("2004-05-12")}]}
+{"id":10671115,"id-copy":10671115,"alias":"Montague","name":"MontagueLangston","user-since":datetime("2007-09-20T00:32:15"),"user-since-copy":datetime("2007-09-20T00:32:15"),"friend-ids":{{18236000,47490167,40246549,25232933,22604487,36974958,44747862,2137180,39244601,39608406,23319330,21166788,21726220,12703943,36564459,8379538,43010567,24538004,173522,6132291,21199763,26285128,2350066}},"employment":[{"organization-name":"Vivaace","start-date":date("2001-01-07")}]}
+{"id":11378911,"id-copy":11378911,"alias":"Courtney","name":"CourtneyBashline","user-since":datetime("2010-10-21T06:13:06"),"user-since-copy":datetime("2010-10-21T06:13:06"),"friend-ids":{{19627264,13699162}},"employment":[{"organization-name":"geomedia","start-date":date("2002-06-21")}]}
+{"id":11297359,"id-copy":11297359,"alias":"Perry","name":"PerryLowe","user-since":datetime("2005-12-28T02:16:57"),"user-since-copy":datetime("2005-12-28T02:16:57"),"friend-ids":{{33439767}},"employment":[{"organization-name":"Ontohothex","start-date":date("2007-08-11"),"end-date":date("2009-05-16")}]}
+{"id":9993001,"id-copy":9993001,"alias":"Herbie","name":"HerbieStall","user-since":datetime("2010-06-14T03:01:11"),"user-since-copy":datetime("2010-06-14T03:01:11"),"friend-ids":{{12003033,40923715,34166285,47927261,638933,17338590}},"employment":[{"organization-name":"Roundhex","start-date":date("2009-07-12")}]}
+{"id":10353946,"id-copy":10353946,"alias":"Cass","name":"CassPirl","user-since":datetime("2010-10-25T21:08:28"),"user-since-copy":datetime("2010-10-25T21:08:28"),"friend-ids":{{43117144,29185875,28524977,4904289,37353728,30484159,40114905,18108320,46098949,30207639}},"employment":[{"organization-name":"Trustbam","start-date":date("2006-06-10")}]}
+{"id":10415575,"id-copy":10415575,"alias":"Amabel","name":"AmabelRoose","user-since":datetime("2011-05-28T10:47:28"),"user-since-copy":datetime("2011-05-28T10:47:28"),"friend-ids":{{22120342,22881927,39043768,27695122,8669783,25973892}},"employment":[{"organization-name":"Streettax","start-date":date("2000-03-14")}]}
+{"id":9190501,"id-copy":9190501,"alias":"Leonardo","name":"LeonardoBarr","user-since":datetime("2008-02-23T14:20:45"),"user-since-copy":datetime("2008-02-23T14:20:45"),"friend-ids":{{24193096,44367993,10307197,20420512,36000544,45069724,42621729,10863302,21701700,7110735,6226449,3269792,12797617,19460642,7357145,27051982,31847212,28691920,382743,11602175,1787538,42283089,19610964}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2001-06-25")}]}
+{"id":11052748,"id-copy":11052748,"alias":"Andriana","name":"AndrianaYonkie","user-since":datetime("2005-05-08T19:49:03"),"user-since-copy":datetime("2005-05-08T19:49:03"),"friend-ids":{{24372868,41932219,14088659,33215970,34384197,16343164,24230672,20937997,23129922,33184913,25421373,12081379,289577,19330874,31625333,34885607,34353478,17694263,34819024,44837603}},"employment":[{"organization-name":"strongex","start-date":date("2003-06-16"),"end-date":date("2008-02-15")}]}
+{"id":9415921,"id-copy":9415921,"alias":"Shad","name":"ShadHaynes","user-since":datetime("2010-01-19T22:19:28"),"user-since-copy":datetime("2010-01-19T22:19:28"),"friend-ids":{{4608515,39839555,31370710,43278478,731705,26523982,15560444,10605444,20229128,41477079,47960417,1744587,35477897,10362849,38394199,24090076,14390416}},"employment":[{"organization-name":"Whitemedia","start-date":date("2010-06-23")}]}
+{"id":10307155,"id-copy":10307155,"alias":"Rhetta","name":"RhettaGarneys","user-since":datetime("2008-03-17T00:33:40"),"user-since-copy":datetime("2008-03-17T00:33:40"),"friend-ids":{{5658375,40536479,47961112,28517297,26103231,32434876,44285321,44471686}},"employment":[{"organization-name":"physcane","start-date":date("2006-06-07"),"end-date":date("2010-10-03")}]}
+{"id":9591646,"id-copy":9591646,"alias":"Hoyt","name":"HoytGilman","user-since":datetime("2011-05-13T07:22:20"),"user-since-copy":datetime("2011-05-13T07:22:20"),"friend-ids":{{11207445}},"employment":[{"organization-name":"U-electrics","start-date":date("2004-04-27")}]}
+{"id":11287666,"id-copy":11287666,"alias":"Darian","name":"DarianHurst","user-since":datetime("2009-05-11T03:33:37"),"user-since-copy":datetime("2009-05-11T03:33:37"),"friend-ids":{{34901893,38687373,30369991,44597588,41413513,24197212,36791517,19949174,23092611,29695794,7024108,25202811,10231736,3754404,15863600,30772236,21615658}},"employment":[{"organization-name":"Coneflex","start-date":date("2012-04-12"),"end-date":date("2012-05-07")}]}
+{"id":10738096,"id-copy":10738096,"alias":"Dori","name":"DoriAlcocke","user-since":datetime("2010-05-21T04:59:08"),"user-since-copy":datetime("2010-05-21T04:59:08"),"friend-ids":{{44039507,40951102,39132038,31982600,46848423,43375356,6188106,3044041,38421537,18640387,21639042,11192576,15659477,360828,26875197,19433881}},"employment":[{"organization-name":"over-it","start-date":date("2008-02-19"),"end-date":date("2011-03-24")}]}
+{"id":11659888,"id-copy":11659888,"alias":"Nannie","name":"NannieWoodworth","user-since":datetime("2006-12-11T15:30:08"),"user-since-copy":datetime("2006-12-11T15:30:08"),"friend-ids":{{30803046,33105462,14783423,5069473,15960335}},"employment":[{"organization-name":"U-electrics","start-date":date("2006-10-12")}]}
+{"id":10132771,"id-copy":10132771,"alias":"Gaenor","name":"GaenorEvans","user-since":datetime("2006-01-23T20:07:34"),"user-since-copy":datetime("2006-01-23T20:07:34"),"friend-ids":{{20344517,47988409,39449785,16775663,20200468}},"employment":[{"organization-name":"Xx-drill","start-date":date("2001-03-17")}]}
+{"id":10473718,"id-copy":10473718,"alias":"Elissa","name":"ElissaStainforth","user-since":datetime("2007-06-20T07:46:54"),"user-since-copy":datetime("2007-06-20T07:46:54"),"friend-ids":{{1645948,612724,46091510,32750261,40622752,10190250,42030152,28645649,27513961}},"employment":[{"organization-name":"Mathtech","start-date":date("2003-06-21"),"end-date":date("2011-09-05")}]}
+{"id":11022826,"id-copy":11022826,"alias":"Virgee","name":"VirgeeHolts","user-since":datetime("2012-01-17T22:54:54"),"user-since-copy":datetime("2012-01-17T22:54:54"),"friend-ids":{{40134062,13624785,23477090,26708578,18967215,21325604,15522457,25873528}},"employment":[{"organization-name":"Dandamace","start-date":date("2004-05-09"),"end-date":date("2010-06-15")}]}
+{"id":9129220,"id-copy":9129220,"alias":"Lessie","name":"LessieGoodman","user-since":datetime("2008-09-01T06:07:35"),"user-since-copy":datetime("2008-09-01T06:07:35"),"friend-ids":{{16418186,35990435,22056439,36479650,36405609,12039460,33551878,10736746,41967761,20046069,8949956,26571267}},"employment":[{"organization-name":"Techitechi","start-date":date("2004-10-23"),"end-date":date("2011-05-08")}]}
+{"id":9929866,"id-copy":9929866,"alias":"Emilie","name":"EmilieJohns","user-since":datetime("2009-10-01T00:51:03"),"user-since-copy":datetime("2009-10-01T00:51:03"),"friend-ids":{{45496950,38109555,46259676,14141368,31720484,35564907,23226721,36026226,34003258,47176035,46593035,5050811,27858647,3784968}},"employment":[{"organization-name":"over-it","start-date":date("2012-08-10"),"end-date":date("2012-08-24")}]}
+{"id":9232504,"id-copy":9232504,"alias":"Lesley","name":"LesleyHujsak","user-since":datetime("2008-07-07T13:30:22"),"user-since-copy":datetime("2008-07-07T13:30:22"),"friend-ids":{{42058063,24501683,26865036,180621}},"employment":[{"organization-name":"Ontohothex","start-date":date("2011-01-04"),"end-date":date("2011-02-07")}]}
+{"id":11454253,"id-copy":11454253,"alias":"Fairy","name":"FairyFoster","user-since":datetime("2007-05-04T11:48:12"),"user-since-copy":datetime("2007-05-04T11:48:12"),"friend-ids":{{15077027,13719617,3663639,16159577,29937764,11018999,36883485,35967804,16558412,19456409,33156277,8763694,9279896}},"employment":[{"organization-name":"Voltlane","start-date":date("2012-07-10")}]}
+{"id":10186180,"id-copy":10186180,"alias":"Mina","name":"MinaGist","user-since":datetime("2012-07-05T21:56:14"),"user-since-copy":datetime("2012-07-05T21:56:14"),"friend-ids":{{12424234,41863508,44607839,36984124,3839840,38458170,41721653,4785194,20595881,13515001}},"employment":[{"organization-name":"whitestreet","start-date":date("2012-07-19")}]}
+{"id":10738477,"id-copy":10738477,"alias":"Kenith","name":"KenithLeichter","user-since":datetime("2012-07-10T15:21:51"),"user-since-copy":datetime("2012-07-10T15:21:51"),"friend-ids":{{}},"employment":[{"organization-name":"Salthex","start-date":date("2006-07-28"),"end-date":date("2009-06-03")}]}
+{"id":10700431,"id-copy":10700431,"alias":"Lessie","name":"LessieRobinson","user-since":datetime("2011-02-03T18:31:41"),"user-since-copy":datetime("2011-02-03T18:31:41"),"friend-ids":{{8174251,46379649,3507858,13269282,38334885,12074283,34128956,46802811,37285621,15203773,17611824,47823053,28609781,31377970,11077457,3771375,27529933,170454,38682017}},"employment":[{"organization-name":"Sanjodax","start-date":date("2002-01-08"),"end-date":date("2006-06-08")}]}
+{"id":10337950,"id-copy":10337950,"alias":"Bibi","name":"BibiCattley","user-since":datetime("2007-11-16T11:08:34"),"user-since-copy":datetime("2007-11-16T11:08:34"),"friend-ids":{{24399247,18391359,18215808,36042641,19360937,2039633,17280287,22159187,31245932,4767019,3299881,12321916,22533524,18760130,31303729,39565694,21606207,8226305,16276064}},"employment":[{"organization-name":"Sumlane","start-date":date("2003-02-25"),"end-date":date("2008-08-20")}]}
+{"id":11697754,"id-copy":11697754,"alias":"Jeanette","name":"JeanetteBullard","user-since":datetime("2005-11-20T09:56:59"),"user-since-copy":datetime("2005-11-20T09:56:59"),"friend-ids":{{22439123,42241829,21396058,6050318,4951741,4940964,22719195,21108984,1496059,41986346,20838301,34979646,19524886,6383593,37747505,26787944,45486736,7537516}},"employment":[{"organization-name":"Inchdox","start-date":date("2006-02-20")}]}
+{"id":11062330,"id-copy":11062330,"alias":"Derick","name":"DerickPennington","user-since":datetime("2008-04-15T11:59:52"),"user-since-copy":datetime("2008-04-15T11:59:52"),"friend-ids":{{26471368,22445928,13709179,16677606,45234923,5601330,16510085,27673980,24365707,42647605,20473849,40448252,37480913,38532114,11022656,799537,38469920,1291033,31503804,29154535,5506108,24609403,35535409,44197253}},"employment":[{"organization-name":"Salthex","start-date":date("2002-09-23")}]}
+{"id":11281576,"id-copy":11281576,"alias":"Louisa","name":"LouisaWheeler","user-since":datetime("2005-01-19T05:34:26"),"user-since-copy":datetime("2005-01-19T05:34:26"),"friend-ids":{{29655724,29204886,24086191,36260050,502778,368888,42853595,40434954,46768026,17096472,33160972,15621748,46246949,14174435,99088,44271646,3676253,11744063,21957250,34611796,32735521,45352911,6097178,3796892}},"employment":[{"organization-name":"Goldcity","start-date":date("2011-03-05")}]}
+{"id":9426244,"id-copy":9426244,"alias":"Lamar","name":"LamarMaugham","user-since":datetime("2005-03-08T17:00:15"),"user-since-copy":datetime("2005-03-08T17:00:15"),"friend-ids":{{36168436,20740167,21922111,32892152,34608833,28621520,40818313,23842558,41275216,36331147,40737858,45983619,14033949,23132425,33634408}},"employment":[{"organization-name":"Sancone","start-date":date("2011-12-24")}]}
+{"id":9967888,"id-copy":9967888,"alias":"Andrea","name":"AndreaBerry","user-since":datetime("2007-05-03T20:18:51"),"user-since-copy":datetime("2007-05-03T20:18:51"),"friend-ids":{{1106859,38049440,23056791,16253206,7727164,19267641,31798723,30455164,24738450,15142413,15111012,3782070,11502933,44299958,30277689,3512757,41960838,7667284,9192069,12267931,34901540,20633036,37186032,1734718}},"employment":[{"organization-name":"ganjalax","start-date":date("2007-04-01"),"end-date":date("2011-09-07")}]}
+{"id":11899576,"id-copy":11899576,"alias":"Raven","name":"RavenAdams","user-since":datetime("2011-12-02T12:46:45"),"user-since-copy":datetime("2011-12-02T12:46:45"),"friend-ids":{{33232775,8985272,34257645,15577012,3749136,36721837,17368752,36931534,30688133,36202643,8373322,34639728,10776563,5758944,19414939,46764976,29704238,38970621,9462886,46724087,29191126,9001393}},"employment":[{"organization-name":"Whitemedia","start-date":date("2003-03-02")}]}
+{"id":10587655,"id-copy":10587655,"alias":"Del","name":"DelLester","user-since":datetime("2006-04-22T06:14:51"),"user-since-copy":datetime("2006-04-22T06:14:51"),"friend-ids":{{41382268,41043817,37053482,27889226,5182442,46241085,39510378,25972421,6234359,2782513,27042023,20476198}},"employment":[{"organization-name":"Rungozoom","start-date":date("2001-10-18")}]}
+{"id":9417499,"id-copy":9417499,"alias":"Wendell","name":"WendellJoyce","user-since":datetime("2011-07-25T14:30:30"),"user-since-copy":datetime("2011-07-25T14:30:30"),"friend-ids":{{10079972,29246113,40533159,31279768,31969044,46120195,35004468,24465042,2702879,44166678,20176481,32056309,38254930,20950061,4687108}},"employment":[{"organization-name":"Labzatron","start-date":date("2006-03-18")}]}
+{"id":9408427,"id-copy":9408427,"alias":"Matt","name":"MattPritchard","user-since":datetime("2008-10-02T15:31:39"),"user-since-copy":datetime("2008-10-02T15:31:39"),"friend-ids":{{3596345,15476624,33857894,13004846,29332890,23638145,43402648,14337754,3290802,10537283,9989868,33400736,43952799,34128983,3090230,12591428,15051691,7239629,10295253,23448932,30507945}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2012-02-05")}]}
+{"id":10735369,"id-copy":10735369,"alias":"Cody","name":"CodySchaeffer","user-since":datetime("2008-07-03T05:27:24"),"user-since-copy":datetime("2008-07-03T05:27:24"),"friend-ids":{{15534779,12333665,10468027,3865324,39537208,16999101,9009757,318331,30123714,10137427,16481424}},"employment":[{"organization-name":"Vaiatech","start-date":date("2009-01-19")}]}
+{"id":9008185,"id-copy":9008185,"alias":"Francene","name":"FranceneZoucks","user-since":datetime("2009-10-18T08:37:00"),"user-since-copy":datetime("2009-10-18T08:37:00"),"friend-ids":{{47321113,34578577,25011033,19259482,6221464,4912987,20361608,27957639,33209653,46928253,37111867,11534180,31643335,39967918,8490889,23713207,28827713,22143989,21710696,3545622,13887489,41557233,26554092}},"employment":[{"organization-name":"Hexsanhex","start-date":date("2004-02-01"),"end-date":date("2011-10-10")}]}
+{"id":11626990,"id-copy":11626990,"alias":"Filiberto","name":"FilibertoFonblanque","user-since":datetime("2006-05-18T07:38:32"),"user-since-copy":datetime("2006-05-18T07:38:32"),"friend-ids":{{41443868,30006940,14137070,14868792,47991977,39513958,32787637,1389727,28607710,21537795,42395037,11730902,25246772,24475669,35786951,32795214}},"employment":[{"organization-name":"whitestreet","start-date":date("2007-08-11")}]}
+{"id":10795960,"id-copy":10795960,"alias":"Hallam","name":"HallamBousum","user-since":datetime("2010-04-23T14:02:10"),"user-since-copy":datetime("2010-04-23T14:02:10"),"friend-ids":{{23447883,39605256,41998325}},"employment":[{"organization-name":"zoomplus","start-date":date("2007-04-18")}]}
+{"id":9418882,"id-copy":9418882,"alias":"Laurine","name":"LaurineCowart","user-since":datetime("2012-06-14T22:26:09"),"user-since-copy":datetime("2012-06-14T22:26:09"),"friend-ids":{{19430214,17084414,12678029,1783933,42580022,26274674,13661281,31117329,19971039,43840305,42672247,17088417,31128028,41009670,16020772}},"employment":[{"organization-name":"Inchex","start-date":date("2002-03-20")}]}
+{"id":9516883,"id-copy":9516883,"alias":"Delsie","name":"DelsieKuster","user-since":datetime("2005-11-20T06:18:01"),"user-since-copy":datetime("2005-11-20T06:18:01"),"friend-ids":{{7211399,31355269,10052966,11255272,11976144,13036749,28228775,3501290,35668522,21064471,47089958,20725508,16768149,39282691,44096922,12469594,8258231,36373387,14994345,32022989,28975684,29840860}},"employment":[{"organization-name":"Rungozoom","start-date":date("2008-11-11"),"end-date":date("2008-03-06")}]}
+{"id":9029377,"id-copy":9029377,"alias":"Boyce","name":"BoyceAnderson","user-since":datetime("2010-12-18T14:17:12"),"user-since-copy":datetime("2010-12-18T14:17:12"),"friend-ids":{{19260027,21449100,35898407,34501982}},"employment":[{"organization-name":"Lexicone","start-date":date("2007-06-25")}]}
+{"id":11878948,"id-copy":11878948,"alias":"Corey","name":"CoreyWarrick","user-since":datetime("2005-05-28T15:18:23"),"user-since-copy":datetime("2005-05-28T15:18:23"),"friend-ids":{{17192577,19646534,44755348,28653064,30539369,15001411,11921646,44450607,33599896,41984600,2187246,8785209,28099595}},"employment":[{"organization-name":"Zimcone","start-date":date("2010-12-07")}]}
+{"id":11333794,"id-copy":11333794,"alias":"Yung","name":"YungNash","user-since":datetime("2010-06-08T17:32:35"),"user-since-copy":datetime("2010-06-08T17:32:35"),"friend-ids":{{11329358,14452899,15459758,31785934,15405998,17431717,36883854,1230831,17690420,45243495,31580409,15264731,10067263,20381783,41240146,2883831,29492394,38409147,35853447,26151247}},"employment":[{"organization-name":"Salthex","start-date":date("2002-06-24"),"end-date":date("2010-03-23")}]}
+{"id":9746482,"id-copy":9746482,"alias":"Ava","name":"AvaEndsley","user-since":datetime("2005-07-05T11:34:59"),"user-since-copy":datetime("2005-07-05T11:34:59"),"friend-ids":{{38589612,37168849,27697487,47869699,7140447,1195276,25105593,46071,5222989,39550451,45838187,8513498,44093597,25194162,11534580,37101502,6417166,23315276,9854625}},"employment":[{"organization-name":"itlab","start-date":date("2011-06-15")}]}
+{"id":11630158,"id-copy":11630158,"alias":"Jewel","name":"JewelPrechtl","user-since":datetime("2008-09-24T10:05:42"),"user-since-copy":datetime("2008-09-24T10:05:42"),"friend-ids":{{17110258,26859370,7070027,19698792,10087924,31999744,35694569,10315290,15006946,25258889,8036893,20721778,31250890,31525573}},"employment":[{"organization-name":"Codetechno","start-date":date("2002-10-09")}]}
+{"id":11003527,"id-copy":11003527,"alias":"Clitus","name":"ClitusDickinson","user-since":datetime("2007-10-18T04:59:18"),"user-since-copy":datetime("2007-10-18T04:59:18"),"friend-ids":{{26264340,47892511,18715043,43994375,42874707,44696774,7281939}},"employment":[{"organization-name":"Inchdox","start-date":date("2006-09-15")}]}
+{"id":11536582,"id-copy":11536582,"alias":"Deon","name":"DeonBickerson","user-since":datetime("2007-05-18T18:12:00"),"user-since-copy":datetime("2007-05-18T18:12:00"),"friend-ids":{{2848304,6359671,29695732,42414044,3277185,17642866,47064497,32240400,43486181,5049864,22831246,9259974,17502793,29955647,6928887,19609966}},"employment":[{"organization-name":"Quadlane","start-date":date("2009-01-20"),"end-date":date("2009-03-12")}]}
+{"id":9958378,"id-copy":9958378,"alias":"Floyd","name":"FloydErrett","user-since":datetime("2006-07-06T02:51:46"),"user-since-copy":datetime("2006-07-06T02:51:46"),"friend-ids":{{38108839,44502073,19244279,45055684,32489890,25184431,34275591,47288414,46973922,28264345,10024409,4791958,40576138,33446414,359486,25595793,25140170,23149057,47032976,4283407}},"employment":[{"organization-name":"Newcom","start-date":date("2001-08-08")}]}
+{"id":11241523,"id-copy":11241523,"alias":"Gareth","name":"GarethFylbrigg","user-since":datetime("2011-01-05T16:02:25"),"user-since-copy":datetime("2011-01-05T16:02:25"),"friend-ids":{{45629812,20113715,13556523,29410246,37849964,33688575,35713924,21492453,32324177,5765413,4491937,1592640,2809253,45152094,36330032,25347157,199553,16471761,16621535,20674800,42682300,11354218,4830164}},"employment":[{"organization-name":"Hot-tech","start-date":date("2005-10-27"),"end-date":date("2005-12-10")}]}
+{"id":11318098,"id-copy":11318098,"alias":"Lucilla","name":"LucillaSteele","user-since":datetime("2006-05-02T12:10:51"),"user-since-copy":datetime("2006-05-02T12:10:51"),"friend-ids":{{43202249,11116520,19404968,23494384,41664359,2459832,21895811,29849475,32963400,24381723,46790616,10343240,43849340,16769526,26104853}},"employment":[{"organization-name":"Unijobam","start-date":date("2009-10-09")}]}
+{"id":10283503,"id-copy":10283503,"alias":"Terrilyn","name":"TerrilynZadovsky","user-since":datetime("2007-06-17T05:40:01"),"user-since-copy":datetime("2007-06-17T05:40:01"),"friend-ids":{{30185148,22395650,3212998,41571861,21336440,41050091}},"employment":[{"organization-name":"Whitemedia","start-date":date("2009-12-14")}]}
+{"id":11293477,"id-copy":11293477,"alias":"Tamzen","name":"TamzenWheeler","user-since":datetime("2006-02-25T23:55:58"),"user-since-copy":datetime("2006-02-25T23:55:58"),"friend-ids":{{}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2002-05-19"),"end-date":date("2011-03-06")}]}
+{"id":11273587,"id-copy":11273587,"alias":"Timmy","name":"TimmyBishop","user-since":datetime("2011-11-08T13:46:03"),"user-since-copy":datetime("2011-11-08T13:46:03"),"friend-ids":{{42987870,44400071,27388256,10579275,12546323,23276512,382419,4466999,8068553,33814105,14872828,35038629,43462816,44037440}},"employment":[{"organization-name":"Doncare","start-date":date("2001-08-08")}]}
+{"id":9331075,"id-copy":9331075,"alias":"Monday","name":"MondayWarrick","user-since":datetime("2012-01-13T06:13:30"),"user-since-copy":datetime("2012-01-13T06:13:30"),"friend-ids":{{27699724,39094128,11014820,44605243,20177679,37579779,35875781,13713739,8882475,37427927,28595578,3788567,31200715,40590973,7630783,36856789,22013865}},"employment":[{"organization-name":"linedexon","start-date":date("2000-04-08")}]}
+{"id":11196118,"id-copy":11196118,"alias":"Carson","name":"CarsonBusk","user-since":datetime("2006-07-23T07:08:34"),"user-since-copy":datetime("2006-07-23T07:08:34"),"friend-ids":{{36454884,31755449,44569587}},"employment":[{"organization-name":"Quadlane","start-date":date("2007-08-13")}]}
+{"id":11674741,"id-copy":11674741,"alias":"Soon","name":"SoonBillimek","user-since":datetime("2009-03-02T12:08:16"),"user-since-copy":datetime("2009-03-02T12:08:16"),"friend-ids":{{26069920,16634341,13963293,27425934,19271848,22444876,42264629,39307655,21118192,27961060,12398172,13202296,23221559,34323488,1588557,42672479,19548482,28266272,6241122,13633490}},"employment":[{"organization-name":"strongex","start-date":date("2006-01-19"),"end-date":date("2011-03-25")}]}
+{"id":10396831,"id-copy":10396831,"alias":"Carman","name":"CarmanElder","user-since":datetime("2011-12-27T21:50:41"),"user-since-copy":datetime("2011-12-27T21:50:41"),"friend-ids":{{41782166,39862540,39100006,45023958,29253172,31208143,12637805,5844876,37296616,20896053,18358082,11068853,5350064,14456765,15758928}},"employment":[{"organization-name":"Hexviafind","start-date":date("2012-05-18"),"end-date":date("2012-07-26")}]}
+{"id":11698384,"id-copy":11698384,"alias":"Bernetta","name":"BernettaFiddler","user-since":datetime("2012-06-20T20:05:46"),"user-since-copy":datetime("2012-06-20T20:05:46"),"friend-ids":{{12203676}},"employment":[{"organization-name":"U-electrics","start-date":date("2000-03-06")}]}
+{"id":11783038,"id-copy":11783038,"alias":"Cecily","name":"CecilyRamsey","user-since":datetime("2011-01-20T23:39:28"),"user-since-copy":datetime("2011-01-20T23:39:28"),"friend-ids":{{30228589,45494315,36823967,2965036,37243358,7140131,8303981,10041948,41439178,24261471,16906521,46190105,45392996,21067630,26632248,44955893}},"employment":[{"organization-name":"jaydax","start-date":date("2000-03-25"),"end-date":date("2010-06-25")}]}
+{"id":11235340,"id-copy":11235340,"alias":"Maurice","name":"MauriceHayhurst","user-since":datetime("2008-12-24T05:11:37"),"user-since-copy":datetime("2008-12-24T05:11:37"),"friend-ids":{{36045307,37144109,37142113,38379399,21011762,30698208,3185430,24698099,39750599,1820110,19740583,5658727,33165497,27066109,20299488,26484094,17984991,9623240,15287433,32468842,34023148,16744372,30389952,40305465}},"employment":[{"organization-name":"Sancone","start-date":date("2011-05-19"),"end-date":date("2011-11-15")}]}
+{"id":10594069,"id-copy":10594069,"alias":"Clinton","name":"ClintonMiller","user-since":datetime("2007-03-12T05:19:19"),"user-since-copy":datetime("2007-03-12T05:19:19"),"friend-ids":{{}},"employment":[{"organization-name":"itlab","start-date":date("2010-06-06")}]}
+{"id":9151357,"id-copy":9151357,"alias":"Clover","name":"CloverTedrow","user-since":datetime("2012-04-04T22:46:03"),"user-since-copy":datetime("2012-04-04T22:46:03"),"friend-ids":{{47959325,11808875,46311157,33138600,15486362,27921017,32586367,24379643,14793815,5841252,22249573,2147304,47811082,40329394,4601822,27977744,45733056}},"employment":[{"organization-name":"Tripplelane","start-date":date("2002-09-07"),"end-date":date("2006-08-04")}]}
+{"id":11821996,"id-copy":11821996,"alias":"Latanya","name":"LatanyaZalack","user-since":datetime("2010-12-07T15:20:09"),"user-since-copy":datetime("2010-12-07T15:20:09"),"friend-ids":{{23521495,43957220,3823403,34033770}},"employment":[{"organization-name":"Hexviafind","start-date":date("2008-04-17")}]}
+{"id":10836430,"id-copy":10836430,"alias":"Kaycee","name":"KayceeCatleay","user-since":datetime("2007-05-18T07:19:02"),"user-since-copy":datetime("2007-05-18T07:19:02"),"friend-ids":{{40568633,44667158,18923311,34987631,29306332,38711535,43999451,3179954,9799980,3451381,23204288,17797804,2164448,16697308,24697554,45250786,10029328,27871642}},"employment":[{"organization-name":"Dandamace","start-date":date("2002-04-25")}]}
+{"id":9809977,"id-copy":9809977,"alias":"Kassandra","name":"KassandraHarding","user-since":datetime("2007-05-01T06:22:22"),"user-since-copy":datetime("2007-05-01T06:22:22"),"friend-ids":{{29945374,38811992,41372042,28714909,16897620,5020268,24134801,26310926,32871167,18787983,47295432,31873694,36300817,42779931,27486692}},"employment":[{"organization-name":"Dandamace","start-date":date("2001-08-26")}]}
+{"id":10777441,"id-copy":10777441,"alias":"Rosaline","name":"RosalineFaast","user-since":datetime("2005-05-23T08:24:59"),"user-since-copy":datetime("2005-05-23T08:24:59"),"friend-ids":{{25088415,36453219,42450810,6845863,23568088,34305276,28849557,41593223,18542045,37652004,9159129,42079452}},"employment":[{"organization-name":"Freshfix","start-date":date("2002-01-04")}]}
+{"id":10073002,"id-copy":10073002,"alias":"Josefa","name":"JosefaNewman","user-since":datetime("2010-10-06T09:28:29"),"user-since-copy":datetime("2010-10-06T09:28:29"),"friend-ids":{{7549910,7287709,24063891,41208589,22325854,16465930,45067165,42784968,26414870,16479308,22681119,40811475,9603161,23525416,15131604,4782290,36997646,35862360,42008502,438438,25913601,39300786,15041382,37410001}},"employment":[{"organization-name":"Keytech","start-date":date("2011-02-05"),"end-date":date("2011-10-24")}]}
+{"id":11488420,"id-copy":11488420,"alias":"Rik","name":"RikSell","user-since":datetime("2011-04-24T10:10:24"),"user-since-copy":datetime("2011-04-24T10:10:24"),"friend-ids":{{37808691,28841986,27850488,28093210,9165013,45941806,5194022,39773028,45473967,44833113,27429268}},"employment":[{"organization-name":"Roundhex","start-date":date("2002-09-23"),"end-date":date("2010-06-23")}]}
+{"id":11092324,"id-copy":11092324,"alias":"Paul","name":"PaulOneal","user-since":datetime("2006-11-20T10:50:19"),"user-since-copy":datetime("2006-11-20T10:50:19"),"friend-ids":{{44707820,20249424,18862268,32895394,29899430}},"employment":[{"organization-name":"Vivaace","start-date":date("2003-01-06")}]}
+{"id":9959077,"id-copy":9959077,"alias":"Josephine","name":"JosephineLauffer","user-since":datetime("2006-12-27T17:33:39"),"user-since-copy":datetime("2006-12-27T17:33:39"),"friend-ids":{{41423014,33024139,26147665,14776436,4726952,12688804}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2001-03-26")}]}
+{"id":9012382,"id-copy":9012382,"alias":"Laureen","name":"LaureenOneal","user-since":datetime("2009-12-10T22:17:58"),"user-since-copy":datetime("2009-12-10T22:17:58"),"friend-ids":{{25012654,4572832,38401260,3015853,42975956,16328675,39626774,26936410,15112607,3302431}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2007-11-10")}]}
+{"id":10193368,"id-copy":10193368,"alias":"Oneida","name":"OneidaEve","user-since":datetime("2005-01-16T07:26:07"),"user-since-copy":datetime("2005-01-16T07:26:07"),"friend-ids":{{46396755,39763353,13661339,5992749,293256,15572483,16775625,21543680}},"employment":[{"organization-name":"Hot-tech","start-date":date("2003-01-12"),"end-date":date("2008-03-22")}]}
+{"id":10591498,"id-copy":10591498,"alias":"Mick","name":"MickVeith","user-since":datetime("2006-02-21T06:58:53"),"user-since-copy":datetime("2006-02-21T06:58:53"),"friend-ids":{{33872347,40692511,18563650}},"employment":[{"organization-name":"Solfix","start-date":date("2011-08-07"),"end-date":date("2011-01-10")}]}
+{"id":11455492,"id-copy":11455492,"alias":"Cymbeline","name":"CymbelineEliza","user-since":datetime("2010-05-03T21:32:10"),"user-since-copy":datetime("2010-05-03T21:32:10"),"friend-ids":{{27738860,21711920,47805508,33507501,22648267,1006513,23617648,20104970,8132761,14963107,19477123}},"employment":[{"organization-name":"Villa-tech","start-date":date("2002-07-25")}]}
+{"id":10779373,"id-copy":10779373,"alias":"Donya","name":"DonyaWegley","user-since":datetime("2012-03-28T01:26:06"),"user-since-copy":datetime("2012-03-28T01:26:06"),"friend-ids":{{24977052,19856115,36795249,7875698,23317261,5916235,17789989,41932923}},"employment":[{"organization-name":"Basecone","start-date":date("2008-10-18")}]}
+{"id":10936273,"id-copy":10936273,"alias":"Hans","name":"HansMench","user-since":datetime("2008-08-08T12:00:48"),"user-since-copy":datetime("2008-08-08T12:00:48"),"friend-ids":{{36800139}},"employment":[{"organization-name":"sonstreet","start-date":date("2005-04-15"),"end-date":date("2009-08-05")}]}
+{"id":9139966,"id-copy":9139966,"alias":"Elwood","name":"ElwoodDavis","user-since":datetime("2009-04-25T20:38:07"),"user-since-copy":datetime("2009-04-25T20:38:07"),"friend-ids":{{28327906,35534034,3278109,20721373,40303614,22594044,3292862,42117489,18133788,31771270,43837818,36567026}},"employment":[{"organization-name":"Inchdox","start-date":date("2004-09-03"),"end-date":date("2011-07-03")}]}
+{"id":11280553,"id-copy":11280553,"alias":"Wendy","name":"WendyClarke","user-since":datetime("2009-08-28T16:53:37"),"user-since-copy":datetime("2009-08-28T16:53:37"),"friend-ids":{{10802559,42649709,8824750,19241403,43339000,23865070,9842110,7051904,39440876,16961992}},"employment":[{"organization-name":"ganjalax","start-date":date("2004-11-15"),"end-date":date("2005-01-15")}]}
+{"id":10177078,"id-copy":10177078,"alias":"Fausto","name":"FaustoLotherington","user-since":datetime("2005-06-23T22:18:16"),"user-since-copy":datetime("2005-06-23T22:18:16"),"friend-ids":{{9405744,13732034}},"employment":[{"organization-name":"Trustbam","start-date":date("2007-12-27")}]}
+{"id":11587666,"id-copy":11587666,"alias":"Kathi","name":"KathiJenner","user-since":datetime("2012-02-20T01:58:30"),"user-since-copy":datetime("2012-02-20T01:58:30"),"friend-ids":{{37156773,10519382,11009989,47883115,13123467,36990044,8554049,47075065,11896169,42580126,43261036,15337748,35985068,44438965,33507413,40063633,32559158,32202309,25536635}},"employment":[{"organization-name":"Inchdox","start-date":date("2001-01-01")}]}
+{"id":11642026,"id-copy":11642026,"alias":"Brenden","name":"BrendenLucy","user-since":datetime("2010-09-18T13:14:17"),"user-since-copy":datetime("2010-09-18T13:14:17"),"friend-ids":{{4037044,13420154,10023579,7611523,10090302,36514218,24369151,10481696,341494}},"employment":[{"organization-name":"Latsonity","start-date":date("2007-07-05")}]}
+{"id":10827610,"id-copy":10827610,"alias":"Madelina","name":"MadelinaCamp","user-since":datetime("2010-06-08T13:22:59"),"user-since-copy":datetime("2010-06-08T13:22:59"),"friend-ids":{{35445385,15924939,7897517,15573537,7234891,46098859,877311,40923818,45519215,27332107,1693386,21101894,35225}},"employment":[{"organization-name":"Hot-tech","start-date":date("2002-10-04")}]}
+{"id":10610356,"id-copy":10610356,"alias":"Jason","name":"JasonGearhart","user-since":datetime("2010-03-05T22:57:20"),"user-since-copy":datetime("2010-03-05T22:57:20"),"friend-ids":{{6967239,47468231,29517365,9206260}},"employment":[{"organization-name":"zoomplus","start-date":date("2012-03-16"),"end-date":date("2012-06-19")}]}
+{"id":9297361,"id-copy":9297361,"alias":"Yasmine","name":"YasmineBullard","user-since":datetime("2006-07-11T23:54:23"),"user-since-copy":datetime("2006-07-11T23:54:23"),"friend-ids":{{27580636,11448774,32271178,9627095,11487349,46595708}},"employment":[{"organization-name":"linedexon","start-date":date("2001-10-06"),"end-date":date("2003-03-05")}]}
+{"id":9107137,"id-copy":9107137,"alias":"Woodrow","name":"WoodrowMueller","user-since":datetime("2012-06-15T04:53:52"),"user-since-copy":datetime("2012-06-15T04:53:52"),"friend-ids":{{39459662,1343459,16606290,21443457,29053037,28244658,27954195,9411908,2059678,24579828,40955404}},"employment":[{"organization-name":"Ransaofan","start-date":date("2002-02-11")}]}
+{"id":9185848,"id-copy":9185848,"alias":"Brendon","name":"BrendonJelliman","user-since":datetime("2008-10-13T17:36:00"),"user-since-copy":datetime("2008-10-13T17:36:00"),"friend-ids":{{12675636,6787931,19218962,12655930}},"employment":[{"organization-name":"Icerunin","start-date":date("2008-06-09"),"end-date":date("2009-10-16")}]}
+{"id":11957011,"id-copy":11957011,"alias":"Frannie","name":"FrannieRoose","user-since":datetime("2007-04-05T18:00:20"),"user-since-copy":datetime("2007-04-05T18:00:20"),"friend-ids":{{9114095,4905395,41862236,21901856,39479601,4025127,1517878,16698416,10853001,18625728,15395201,17825510,40384476,18779630,1832149,41381869,40010653,21121933,18598397,12806945,11465558}},"employment":[{"organization-name":"Y-geohex","start-date":date("2006-12-22")}]}
+{"id":10493269,"id-copy":10493269,"alias":"Anya","name":"AnyaWoodward","user-since":datetime("2009-03-08T07:08:04"),"user-since-copy":datetime("2009-03-08T07:08:04"),"friend-ids":{{2357333}},"employment":[{"organization-name":"Zuncan","start-date":date("2001-05-04")}]}
+{"id":11348449,"id-copy":11348449,"alias":"Domitila","name":"DomitilaPolson","user-since":datetime("2009-09-24T21:31:17"),"user-since-copy":datetime("2009-09-24T21:31:17"),"friend-ids":{{46755392,24913792,47792230,2451253,10548653,3083052,20700516,15133622,17284439,40871072,6444103,44749243,45289097,19631062,8873017,6262067,4742977,672148,19303779}},"employment":[{"organization-name":"Latsonity","start-date":date("2005-06-03")}]}
+{"id":11089501,"id-copy":11089501,"alias":"Antonette","name":"AntonetteBrandenburg","user-since":datetime("2010-01-02T05:42:44"),"user-since-copy":datetime("2010-01-02T05:42:44"),"friend-ids":{{18054329,21707156,1570987,17610288,32279976,10880989,37459189,9057880,46495123,29331373,20615029,22282366,22218648,15950453,30669615,46097959,16640911,15896647}},"employment":[{"organization-name":"Lexicone","start-date":date("2004-10-01"),"end-date":date("2009-02-20")}]}
+{"id":10840990,"id-copy":10840990,"alias":"Libby","name":"LibbyHayhurst","user-since":datetime("2009-10-28T22:52:04"),"user-since-copy":datetime("2009-10-28T22:52:04"),"friend-ids":{{32146321,47850956,42432761,28856789,18595962,23408710,37015546}},"employment":[{"organization-name":"Lexicone","start-date":date("2003-01-20")}]}
+{"id":10975810,"id-copy":10975810,"alias":"Davin","name":"DavinKifer","user-since":datetime("2005-08-19T20:23:07"),"user-since-copy":datetime("2005-08-19T20:23:07"),"friend-ids":{{20162027,7842505,3191764,11487126,44589086,14959953,18826364,18917713,37717273,24319173,1393081,19608709,47932966,37681921,47734310,21616345,21035793,9650227,43642280,21890130,17249802,27944839}},"employment":[{"organization-name":"Alphadax","start-date":date("2011-12-17"),"end-date":date("2011-12-01")}]}
+{"id":9295696,"id-copy":9295696,"alias":"Margaux","name":"MargauxPerkins","user-since":datetime("2012-05-23T04:28:13"),"user-since-copy":datetime("2012-05-23T04:28:13"),"friend-ids":{{23713491,4271158,27340057,7815427,14232017,22868851,2293397,24147381,11816307,16597552,47120663,40746124,9777479,18134957,39193317,19755909,42252346}},"employment":[{"organization-name":"Doublezone","start-date":date("2006-02-17"),"end-date":date("2007-05-06")}]}
+{"id":10786129,"id-copy":10786129,"alias":"Ardelle","name":"ArdelleHoopengarner","user-since":datetime("2012-05-27T08:36:37"),"user-since-copy":datetime("2012-05-27T08:36:37"),"friend-ids":{{44854493,13697746,8918104,22353878,46059542,23393155,37374548,1531344,31554501,30390740,10076243,19028830,46174212,4991316,30988902,6717568}},"employment":[{"organization-name":"Ontohothex","start-date":date("2011-03-08")}]}
+{"id":9341008,"id-copy":9341008,"alias":"Gus","name":"GusGearhart","user-since":datetime("2012-05-23T13:19:57"),"user-since-copy":datetime("2012-05-23T13:19:57"),"friend-ids":{{20124243,19722425,20605718,21833401,18276801,46184199,40454562,22828817,44122338,4485860,34209581,19783645,44454238,1353350,37958534,33547730,2456119,3023314,44828467,46655836,33144170,16864855,41938662}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2002-07-07")}]}
+{"id":10380031,"id-copy":10380031,"alias":"Otha","name":"OthaHaines","user-since":datetime("2005-08-08T04:10:50"),"user-since-copy":datetime("2005-08-08T04:10:50"),"friend-ids":{{2710866,28894512,36379679,32545673,38671874,16746916,39103475,19783615,17514492,42617267,7461114,17712393,43474200,3806350,5065542,35722940}},"employment":[{"organization-name":"zoomplus","start-date":date("2011-05-07")}]}
+{"id":11989660,"id-copy":11989660,"alias":"Rolland","name":"RollandGarneis","user-since":datetime("2008-09-16T19:54:32"),"user-since-copy":datetime("2008-09-16T19:54:32"),"friend-ids":{{30959592,6160903,27316367,6518756,23008668,36942525,39489068,8710310,17726852,72593,15440937,4901728,28916846,38257093,28414859,8857050}},"employment":[{"organization-name":"Redelectronics","start-date":date("2008-02-11")}]}
+{"id":10128076,"id-copy":10128076,"alias":"Parker","name":"ParkerHutton","user-since":datetime("2011-06-05T03:46:01"),"user-since-copy":datetime("2011-06-05T03:46:01"),"friend-ids":{{24818185,42512828,22798434,38901116,12147430,47942796,34742031,7142883,11882526,16055416,3892909,12824325,13378363,34281637,15457426,24092146,27678834,15804956}},"employment":[{"organization-name":"jaydax","start-date":date("2007-04-12"),"end-date":date("2009-05-09")}]}
+{"id":9329272,"id-copy":9329272,"alias":"Nonie","name":"NonieStafford","user-since":datetime("2005-10-01T21:12:24"),"user-since-copy":datetime("2005-10-01T21:12:24"),"friend-ids":{{42745071,14744035,37742648,31179205,28520118,32828516,2726599,1667680}},"employment":[{"organization-name":"Trustbam","start-date":date("2004-06-21")}]}
+{"id":10899544,"id-copy":10899544,"alias":"Valentine","name":"ValentineFisher","user-since":datetime("2008-07-04T14:36:11"),"user-since-copy":datetime("2008-07-04T14:36:11"),"friend-ids":{{26471524,781270,17136010,12943313,42125653,40372131}},"employment":[{"organization-name":"Technohow","start-date":date("2008-09-02"),"end-date":date("2008-01-21")}]}
+{"id":9829834,"id-copy":9829834,"alias":"Darryl","name":"DarrylSullivan","user-since":datetime("2011-07-24T00:12:33"),"user-since-copy":datetime("2011-07-24T00:12:33"),"friend-ids":{{8297654,6071837,27236382,4657522,9035310,40427605,2360931,19796421,7301200,1264845,12653555,27518516}},"employment":[{"organization-name":"zoomplus","start-date":date("2005-01-18"),"end-date":date("2010-05-20")}]}
+{"id":9937957,"id-copy":9937957,"alias":"Corey","name":"CoreyTaggart","user-since":datetime("2005-11-25T16:13:03"),"user-since-copy":datetime("2005-11-25T16:13:03"),"friend-ids":{{40105038,9364511,47362703,1876955,3505769,41708385,36179634,7022850}},"employment":[{"organization-name":"Alphadax","start-date":date("2005-09-24")}]}
+{"id":10858339,"id-copy":10858339,"alias":"Eugenio","name":"EugenioLangston","user-since":datetime("2006-06-14T22:24:18"),"user-since-copy":datetime("2006-06-14T22:24:18"),"friend-ids":{{18107191,19162062,26048227,16199255,32644324,3917262,38994370,36221435,34919041}},"employment":[{"organization-name":"Quadlane","start-date":date("2011-02-03")}]}
+{"id":10858909,"id-copy":10858909,"alias":"Kiley","name":"KileyCoates","user-since":datetime("2011-02-03T03:12:41"),"user-since-copy":datetime("2011-02-03T03:12:41"),"friend-ids":{{47990206,29775839,33872749,38952297,38802567,38822660,12420330,18852873,30468156,29085185,2660660,28283210,6711584,35851765,31124383,39930865,18329720}},"employment":[{"organization-name":"physcane","start-date":date("2006-09-22")}]}
+{"id":11059435,"id-copy":11059435,"alias":"Lucina","name":"LucinaDurstine","user-since":datetime("2007-04-14T19:19:23"),"user-since-copy":datetime("2007-04-14T19:19:23"),"friend-ids":{{18983436,36225185,42601602,22134709,20671612}},"employment":[{"organization-name":"Medflex","start-date":date("2009-06-15")}]}
+{"id":10479190,"id-copy":10479190,"alias":"Carmine","name":"CarmineMortland","user-since":datetime("2011-06-18T02:57:13"),"user-since-copy":datetime("2011-06-18T02:57:13"),"friend-ids":{{36090597,35550849,19614765,34665409,7740163,12824683,12997403,32586142,10137983,44900811,30392212,43177710,47792212}},"employment":[{"organization-name":"jaydax","start-date":date("2002-02-23")}]}
+{"id":10474273,"id-copy":10474273,"alias":"Juliana","name":"JulianaLing","user-since":datetime("2005-05-04T20:58:12"),"user-since-copy":datetime("2005-05-04T20:58:12"),"friend-ids":{{8881381,34113161,15553599,40081858,12450920,42147178,568875,11891228,13309462,39127120,34765111,19162279,29505162,891909,33485893,25658561,36146447,37027867,39396759}},"employment":[{"organization-name":"Mathtech","start-date":date("2008-03-03")}]}
+{"id":10541299,"id-copy":10541299,"alias":"Derrick","name":"DerrickLarson","user-since":datetime("2009-09-04T09:42:12"),"user-since-copy":datetime("2009-09-04T09:42:12"),"friend-ids":{{39544341,9620318,40218798,34927427,28533075,44505091,29066144,31724565,46052997,3011652,24709291,24805644,41125094,14186985,24967210,32420881,31162758,2356654,11854218,47933360,9668743,26801113}},"employment":[{"organization-name":"Techitechi","start-date":date("2000-11-25")}]}
+{"id":11490220,"id-copy":11490220,"alias":"Ernestine","name":"ErnestineWheeler","user-since":datetime("2005-01-27T23:36:35"),"user-since-copy":datetime("2005-01-27T23:36:35"),"friend-ids":{{12995063,40353122,11162426,42762839,9575788,7725738,29883894,48002015,5516807,12731814,33203496,44912740,19681146,5849671,4702317}},"employment":[{"organization-name":"Roundhex","start-date":date("2008-06-16"),"end-date":date("2011-12-01")}]}
+{"id":9005038,"id-copy":9005038,"alias":"Anabel","name":"AnabelWheeler","user-since":datetime("2006-12-12T13:40:23"),"user-since-copy":datetime("2006-12-12T13:40:23"),"friend-ids":{{18713256,35193719,42245821,37249622,12210708,15557948,467039,43997520,45171035,43682410,47884198,43102086,39620955,36438278,42976932,11158113,21543594,9861181,36944403,47928849,29593861,37897057,42360015,27956902}},"employment":[{"organization-name":"strongex","start-date":date("2000-03-21")}]}
+{"id":10930153,"id-copy":10930153,"alias":"Liliana","name":"LilianaGoodman","user-since":datetime("2009-06-22T20:57:17"),"user-since-copy":datetime("2009-06-22T20:57:17"),"friend-ids":{{4302195,1569986,5108357,40772631,30372008,36454077,26878227,10958250,46069776,4779188,46627230,47074148,25489453,24956443,31679399,21835639,42097220,35662047,6354581,34282348,13473927}},"employment":[{"organization-name":"Ontohothex","start-date":date("2008-10-25")}]}
+{"id":10540441,"id-copy":10540441,"alias":"Albert","name":"AlbertBasinger","user-since":datetime("2007-05-12T06:03:38"),"user-since-copy":datetime("2007-05-12T06:03:38"),"friend-ids":{{36392592,35815177,22050314,45279196,15405747,33802667,44081359,2027267,47159697,20007080}},"employment":[{"organization-name":"Hexviafind","start-date":date("2002-10-04"),"end-date":date("2005-08-17")}]}
+{"id":11131138,"id-copy":11131138,"alias":"Maximillian","name":"MaximillianSloan","user-since":datetime("2009-12-26T13:02:42"),"user-since-copy":datetime("2009-12-26T13:02:42"),"friend-ids":{{4007900,16474597,36917058,46709116,35833748,7074328,6125321,40646485,23690629,3251896,3973740,17863849,9389737,26501803,4207105}},"employment":[{"organization-name":"Ronholdings","start-date":date("2010-10-16")}]}
+{"id":10422310,"id-copy":10422310,"alias":"Edmundo","name":"EdmundoShaw","user-since":datetime("2012-07-02T11:10:15"),"user-since-copy":datetime("2012-07-02T11:10:15"),"friend-ids":{{4235436,16381036,12579129,43280339,16455681,28445764,10796826,28577255,15173785,47982248,11990921,2093558,6244669,4830927,34859603,22246754,45142656}},"employment":[{"organization-name":"Hexsanhex","start-date":date("2011-01-27")}]}
+{"id":9035089,"id-copy":9035089,"alias":"Marylyn","name":"MarylynSteele","user-since":datetime("2005-04-24T04:55:25"),"user-since-copy":datetime("2005-04-24T04:55:25"),"friend-ids":{{4250473,16568038,10872744,32513859,37267973,2225211,45148996,1080441,13013464,10394988,3316854,8183563,228753,6849521,8786964,21073526}},"employment":[{"organization-name":"Kongreen","start-date":date("2011-02-11"),"end-date":date("2011-10-08")}]}
+{"id":9783310,"id-copy":9783310,"alias":"Basil","name":"BasilLangston","user-since":datetime("2005-06-10T11:35:51"),"user-since-copy":datetime("2005-06-10T11:35:51"),"friend-ids":{{21087606,17287729,8132136,17055542,5795845,41180261,10977404,29700430,47047119,358942,29290990,19557422,35447157,33135473,36720866,39510564}},"employment":[{"organization-name":"Coneflex","start-date":date("2000-05-11"),"end-date":date("2000-03-09")}]}
+{"id":9910003,"id-copy":9910003,"alias":"Arline","name":"ArlineElinor","user-since":datetime("2012-07-20T16:57:36"),"user-since-copy":datetime("2012-07-20T16:57:36"),"friend-ids":{{34121202,19342891,45323168,17272278,6471047,3726738,48003127,32423724,38588754,44816854,13688032,12876442}},"employment":[{"organization-name":"Quoline","start-date":date("2009-07-19"),"end-date":date("2009-04-17")}]}
+{"id":11259028,"id-copy":11259028,"alias":"Linsay","name":"LinsayBranson","user-since":datetime("2011-04-28T08:49:14"),"user-since-copy":datetime("2011-04-28T08:49:14"),"friend-ids":{{24222662,814967,16722114,24161306,31611,2964110,4912379}},"employment":[{"organization-name":"tresline","start-date":date("2006-05-18"),"end-date":date("2006-12-16")}]}
+{"id":11666128,"id-copy":11666128,"alias":"Mathilda","name":"MathildaBurris","user-since":datetime("2006-01-04T14:30:09"),"user-since-copy":datetime("2006-01-04T14:30:09"),"friend-ids":{{21229678,40152290,2867638,27694777,34054129,47727334,39805693,9084777,37744206,47011794,2190990,19109454}},"employment":[{"organization-name":"Solophase","start-date":date("2007-09-14"),"end-date":date("2007-03-17")}]}
+{"id":10301008,"id-copy":10301008,"alias":"Edgardo","name":"EdgardoWheeler","user-since":datetime("2012-04-27T03:11:16"),"user-since-copy":datetime("2012-04-27T03:11:16"),"friend-ids":{{44525957,2368018}},"employment":[{"organization-name":"Zuncan","start-date":date("2004-07-02"),"end-date":date("2009-04-13")}]}
+{"id":9682723,"id-copy":9682723,"alias":"Rick","name":"RickEisaman","user-since":datetime("2011-01-04T04:42:13"),"user-since-copy":datetime("2011-01-04T04:42:13"),"friend-ids":{{843458,40779817,24515616,9016765,37332064,2164822,45832315,27168757,43771964,46638388,43667809}},"employment":[{"organization-name":"Voltlane","start-date":date("2004-08-13"),"end-date":date("2011-04-11")}]}
+{"id":10662082,"id-copy":10662082,"alias":"Colbert","name":"ColbertFylbrigg","user-since":datetime("2005-04-09T18:04:54"),"user-since-copy":datetime("2005-04-09T18:04:54"),"friend-ids":{{25358191,27442450,16828484,16821866,7010321,35271072,32519925,15521808,35168957,36812363,18888093,45727757,30009499,31505405,27925036,47549214,20290733,18290760,36238437,32377676}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2008-04-02")}]}
+{"id":11899861,"id-copy":11899861,"alias":"Jacki","name":"JackiLeach","user-since":datetime("2009-01-07T13:33:40"),"user-since-copy":datetime("2009-01-07T13:33:40"),"friend-ids":{{17554995,17598007,2855045,4108843,47202404,42565398,45821410,32619673,7988594,7631349,20552170,13116128,14526615,17916951,43018507,18114607}},"employment":[{"organization-name":"silfind","start-date":date("2003-06-24")}]}
+{"id":11592799,"id-copy":11592799,"alias":"Booker","name":"BookerBurkett","user-since":datetime("2008-07-19T14:13:28"),"user-since-copy":datetime("2008-07-19T14:13:28"),"friend-ids":{{8693431,28970363,8276536,42506445,20113337,40761495}},"employment":[{"organization-name":"Newcom","start-date":date("2010-10-26"),"end-date":date("2010-11-15")}]}
+{"id":9683656,"id-copy":9683656,"alias":"Antone","name":"AntoneMays","user-since":datetime("2006-07-24T22:48:29"),"user-since-copy":datetime("2006-07-24T22:48:29"),"friend-ids":{{11275116,40325672,41154035,8987353,31187312,11505721,11584703,42743337,23225356,8653923}},"employment":[{"organization-name":"overtech","start-date":date("2011-06-12")}]}
+{"id":10135477,"id-copy":10135477,"alias":"Jasmine","name":"JasmineEva","user-since":datetime("2009-04-03T11:48:27"),"user-since-copy":datetime("2009-04-03T11:48:27"),"friend-ids":{{3776073}},"employment":[{"organization-name":"Basecone","start-date":date("2000-11-14"),"end-date":date("2001-05-19")}]}
+{"id":11811196,"id-copy":11811196,"alias":"Levi","name":"LeviVeith","user-since":datetime("2010-04-28T03:02:38"),"user-since-copy":datetime("2010-04-28T03:02:38"),"friend-ids":{{24907725,35390929,34837809,5881290,28179492,44686412,32544180,20478414,15685375,8767940,7295427}},"employment":[{"organization-name":"U-electrics","start-date":date("2004-09-01")}]}
+{"id":10925071,"id-copy":10925071,"alias":"Gil","name":"GilFocell","user-since":datetime("2005-11-08T20:28:01"),"user-since-copy":datetime("2005-11-08T20:28:01"),"friend-ids":{{9416716,42743353,43396785,44271346,32924780,44752785,19741326,39315503,25154503,29170056,15457515,14764269,47861907,15230067,15326613,6336542,44127013,1048087,34624221,19951452,12778135}},"employment":[{"organization-name":"Zuncan","start-date":date("2009-01-07")}]}
+{"id":10211827,"id-copy":10211827,"alias":"Fanny","name":"FannyHarrold","user-since":datetime("2010-08-28T09:57:52"),"user-since-copy":datetime("2010-08-28T09:57:52"),"friend-ids":{{4061493,30492642,8550070,34805906,5798646,39169853,45190690,34218456,3758565,18038216}},"employment":[{"organization-name":"Lexitechno","start-date":date("2008-10-14"),"end-date":date("2008-05-18")}]}
+{"id":11179192,"id-copy":11179192,"alias":"Derren","name":"DerrenClose","user-since":datetime("2008-04-28T09:18:19"),"user-since-copy":datetime("2008-04-28T09:18:19"),"friend-ids":{{43947479,30154889,10673575,8056171,28691242,22881730,15291446,7331632,32819016,35194153}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2002-10-24"),"end-date":date("2006-08-12")}]}
+{"id":10962466,"id-copy":10962466,"alias":"Zoey","name":"ZoeyCady","user-since":datetime("2012-07-15T20:02:23"),"user-since-copy":datetime("2012-07-15T20:02:23"),"friend-ids":{{12726157,268799,29381478,15699674,1150948,8000369,41608951,11382366,770690,25889785,37815043,40437016,38679636,32956275,34853801}},"employment":[{"organization-name":"Villa-dox","start-date":date("2012-05-08")}]}
+{"id":10867624,"id-copy":10867624,"alias":"Fredric","name":"FredricKimmons","user-since":datetime("2005-05-14T23:08:00"),"user-since-copy":datetime("2005-05-14T23:08:00"),"friend-ids":{{25574899,26822046,3408550,40738004,3813112,33045116,9229839,28557630,36781441,23585776}},"employment":[{"organization-name":"Dandamace","start-date":date("2010-02-25"),"end-date":date("2011-07-06")}]}
+{"id":10394632,"id-copy":10394632,"alias":"Marlin","name":"MarlinLogue","user-since":datetime("2011-08-28T14:57:40"),"user-since-copy":datetime("2011-08-28T14:57:40"),"friend-ids":{{45667126}},"employment":[{"organization-name":"goldendexon","start-date":date("2004-07-03"),"end-date":date("2009-05-09")}]}
+{"id":9025786,"id-copy":9025786,"alias":"Terrance","name":"TerranceFinlay","user-since":datetime("2009-12-28T02:19:23"),"user-since-copy":datetime("2009-12-28T02:19:23"),"friend-ids":{{45324679,13507068,46678304,37010727,44866157,12584675,34305776,14467180,37751377,2448873,32584169,14120838,8902593,31955437,13436805}},"employment":[{"organization-name":"Unijobam","start-date":date("2012-02-19"),"end-date":date("2012-07-25")}]}
+{"id":10637896,"id-copy":10637896,"alias":"Hiram","name":"HiramRohtin","user-since":datetime("2006-11-05T14:44:03"),"user-since-copy":datetime("2006-11-05T14:44:03"),"friend-ids":{{1387663,11367203,24828245}},"employment":[{"organization-name":"Sancone","start-date":date("2012-02-18"),"end-date":date("2012-02-12")}]}
+{"id":11791471,"id-copy":11791471,"alias":"Robt","name":"RobtChristman","user-since":datetime("2009-08-08T21:01:18"),"user-since-copy":datetime("2009-08-08T21:01:18"),"friend-ids":{{9265036,17976405,32435071,7236713,21936800,42691957,35478695,40052609,14063303,43864025,1254074,39237113,11307270,37061951,17360733,21102633,21364546,35445000,44857867}},"employment":[{"organization-name":"Tripplelane","start-date":date("2008-01-02"),"end-date":date("2010-05-19")}]}
+{"id":11022889,"id-copy":11022889,"alias":"Aubrey","name":"AubreyMccallum","user-since":datetime("2009-08-17T02:42:54"),"user-since-copy":datetime("2009-08-17T02:42:54"),"friend-ids":{{22265320,4304911,3403321,20791603,31499855,22278594,14580040,31651270,14509751,13733306,10947101,7713960}},"employment":[{"organization-name":"highfax","start-date":date("2001-01-11")}]}
+{"id":11934781,"id-copy":11934781,"alias":"Titus","name":"TitusGertraht","user-since":datetime("2011-05-02T12:41:28"),"user-since-copy":datetime("2011-05-02T12:41:28"),"friend-ids":{{32699552,17016611,46281182,32515791,12860342,22463323,33042577,4477908,37152051,5462628,45666108,42424199,44831639,44546969,30686685,40580034}},"employment":[{"organization-name":"Coneflex","start-date":date("2000-04-16")}]}
+{"id":9646474,"id-copy":9646474,"alias":"Lilac","name":"LilacWoodworth","user-since":datetime("2009-12-17T02:42:51"),"user-since-copy":datetime("2009-12-17T02:42:51"),"friend-ids":{{47784123,45348808,36392712,9381262,10215254,1461251,23038092,44549001,39097217,41152823,31758517,19401493,39964393,46307214,41683224,39011968,5014398,482179,3789628,46257340,36041029,10903757,5980810,31935548}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2004-10-25"),"end-date":date("2005-05-04")}]}
+{"id":10469980,"id-copy":10469980,"alias":"Rosalynne","name":"RosalynneZalack","user-since":datetime("2012-03-07T10:12:20"),"user-since-copy":datetime("2012-03-07T10:12:20"),"friend-ids":{{46118617,27264184,8045697,30832992,47861079,24266748,10689886,14799850,1178687,39540720,17568852,24394222,10078451,4748570,47808632,35277954,8802885,13747535,22203533,42065169,19096770,14087466,45753492}},"employment":[{"organization-name":"Vaiatech","start-date":date("2006-04-15"),"end-date":date("2010-07-14")}]}
+{"id":10811875,"id-copy":10811875,"alias":"Giovanni","name":"GiovanniWarner","user-since":datetime("2009-05-28T04:20:11"),"user-since-copy":datetime("2009-05-28T04:20:11"),"friend-ids":{{8005226,21432611,4037183,40486007,40666777,24385549,3686021,12188144,33646224,46365125,44351069,34408172,35904411,4322876,18767645,10007322}},"employment":[{"organization-name":"Salthex","start-date":date("2005-07-18"),"end-date":date("2011-10-24")}]}
+{"id":11822506,"id-copy":11822506,"alias":"Jerrold","name":"JerroldEwing","user-since":datetime("2010-08-27T22:34:36"),"user-since-copy":datetime("2010-08-27T22:34:36"),"friend-ids":{{}},"employment":[{"organization-name":"Sanjodax","start-date":date("2007-03-21"),"end-date":date("2008-04-26")}]}
+{"id":10469071,"id-copy":10469071,"alias":"Apryl","name":"AprylWatson","user-since":datetime("2006-10-03T08:37:12"),"user-since-copy":datetime("2006-10-03T08:37:12"),"friend-ids":{{4517575,34635569,1199146}},"employment":[{"organization-name":"Vivaace","start-date":date("2001-02-01"),"end-date":date("2007-09-01")}]}
+{"id":10948315,"id-copy":10948315,"alias":"Munro","name":"MunroDiegel","user-since":datetime("2006-11-24T10:55:36"),"user-since-copy":datetime("2006-11-24T10:55:36"),"friend-ids":{{46912879,47760999,8438850,12005776,7286415,41598308,42462653,2040525,8432844,39644931}},"employment":[{"organization-name":"Zuncan","start-date":date("2011-08-08"),"end-date":date("2011-09-27")}]}
+{"id":9046852,"id-copy":9046852,"alias":"Mauro","name":"MauroChase","user-since":datetime("2011-04-18T20:18:58"),"user-since-copy":datetime("2011-04-18T20:18:58"),"friend-ids":{{28268506,13880377,18637778,27129860,47146036,23136396,34534506,23274864,38781071,9644011,34754620,45178277,33832472,31871984,47201051,42153557,12418584,37615805,35474951,29273401,4845352,18687033}},"employment":[{"organization-name":"Salthex","start-date":date("2012-05-14"),"end-date":date("2012-06-25")}]}
+{"id":9545461,"id-copy":9545461,"alias":"Sandra","name":"SandraFea","user-since":datetime("2005-12-09T14:40:28"),"user-since-copy":datetime("2005-12-09T14:40:28"),"friend-ids":{{28976045}},"employment":[{"organization-name":"Voltbam","start-date":date("2012-02-02")}]}
+{"id":11515477,"id-copy":11515477,"alias":"Kassandra","name":"KassandraByers","user-since":datetime("2005-05-24T10:27:06"),"user-since-copy":datetime("2005-05-24T10:27:06"),"friend-ids":{{23979652,25789717,7769765,30747470,30667193,22447318,42934938,24601934,31839813,18960206,30913033,39059809,18213877,3731518,10573130,37902022}},"employment":[{"organization-name":"over-it","start-date":date("2004-01-13")}]}
+{"id":11428300,"id-copy":11428300,"alias":"Major","name":"MajorGreenawalt","user-since":datetime("2006-12-02T06:43:13"),"user-since-copy":datetime("2006-12-02T06:43:13"),"friend-ids":{{8021918,4810021,34724015,45030049,36575685,44527472}},"employment":[{"organization-name":"itlab","start-date":date("2002-04-17")}]}
+{"id":9354127,"id-copy":9354127,"alias":"Seymour","name":"SeymourFlick","user-since":datetime("2011-06-17T06:00:11"),"user-since-copy":datetime("2011-06-17T06:00:11"),"friend-ids":{{7662170,25563062,18178019,32667220,12254954,7192061,18829113,8959008,1692176,28852587,17130396,12781461,4083182,11054115,10558861,13876198}},"employment":[{"organization-name":"Kongreen","start-date":date("2007-11-23")}]}
+{"id":10322398,"id-copy":10322398,"alias":"Alanna","name":"AlannaBollinger","user-since":datetime("2008-09-01T20:05:18"),"user-since-copy":datetime("2008-09-01T20:05:18"),"friend-ids":{{4294902,42664964}},"employment":[{"organization-name":"Fixdintex","start-date":date("2009-10-08"),"end-date":date("2011-09-26")}]}
+{"id":10207636,"id-copy":10207636,"alias":"Stewart","name":"StewartHamilton","user-since":datetime("2008-11-06T21:44:47"),"user-since-copy":datetime("2008-11-06T21:44:47"),"friend-ids":{{25417411,7322723,13495699,47274757,44964322,4993843,36429109,11904558,18759232,45446850,40537858,40487724,36200691,6846408,7421262,2225424,12997194}},"employment":[{"organization-name":"Ontohothex","start-date":date("2012-08-28"),"end-date":date("2012-08-29")}]}
+{"id":9997456,"id-copy":9997456,"alias":"Micah","name":"MicahRogers","user-since":datetime("2008-03-01T05:53:42"),"user-since-copy":datetime("2008-03-01T05:53:42"),"friend-ids":{{17761154,33509079,36866187,24618619,7048673,18747407,31947241,33710255,40699565,22334622,24425777,19450074,39309621,4464803,15881946,35888289,10539684,17175942,20754578,27045156,14301629,19478576}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2011-02-16")}]}
+{"id":11918764,"id-copy":11918764,"alias":"Jamison","name":"JamisonKnight","user-since":datetime("2012-02-28T12:46:09"),"user-since-copy":datetime("2012-02-28T12:46:09"),"friend-ids":{{5296309,37783012,18620712,8255206,10270999,47361618,39691488,33528430,22926601,12751125,34000354,32638692,19461108,9760202,30157968,265361,24683869,19612648,29021437,40094162}},"employment":[{"organization-name":"Dandamace","start-date":date("2003-04-16"),"end-date":date("2011-08-28")}]}
+{"id":10937893,"id-copy":10937893,"alias":"Katheleen","name":"KatheleenEisenmann","user-since":datetime("2012-06-17T05:15:08"),"user-since-copy":datetime("2012-06-17T05:15:08"),"friend-ids":{{30129247,865896,35091601,19852276,43238329,46057691,30405091,3723169,6577863,12648596,34726408,19178848,18365491,28604299,29242262,12826786,19046213,23320700,9318080,35996590,24812162,9639554,33615920,6507511}},"employment":[{"organization-name":"Groovetex","start-date":date("2006-07-26")}]}
+{"id":10884241,"id-copy":10884241,"alias":"Anamaria","name":"AnamariaMoon","user-since":datetime("2005-03-28T11:38:17"),"user-since-copy":datetime("2005-03-28T11:38:17"),"friend-ids":{{21445295,42154978,41608378,3406391,26013137,45437958,22377352,26150886,25726611,31834547,17506680,22932063,16700407,22939810,152978,45307280,42212660,30124140,9494103,35217706,41538534,26586744,26538590}},"employment":[{"organization-name":"Newphase","start-date":date("2011-09-10"),"end-date":date("2011-02-06")}]}
+{"id":9521401,"id-copy":9521401,"alias":"Donnette","name":"DonnetteFaust","user-since":datetime("2012-03-22T09:38:14"),"user-since-copy":datetime("2012-03-22T09:38:14"),"friend-ids":{{25050925}},"employment":[{"organization-name":"Mathtech","start-date":date("2008-08-20"),"end-date":date("2009-07-09")}]}
+{"id":9042022,"id-copy":9042022,"alias":"Fran","name":"FranIronmonger","user-since":datetime("2006-05-22T03:51:10"),"user-since-copy":datetime("2006-05-22T03:51:10"),"friend-ids":{{38546356,31805246}},"employment":[{"organization-name":"Tranzap","start-date":date("2002-06-06")}]}
+{"id":11874358,"id-copy":11874358,"alias":"Rachyl","name":"RachylOmara","user-since":datetime("2008-05-19T19:05:44"),"user-since-copy":datetime("2008-05-19T19:05:44"),"friend-ids":{{17070163,39951748,9940832,6714785,4963198,17121038,29997771,21420071,3672434,37974288}},"employment":[{"organization-name":"Fixdintex","start-date":date("2001-11-11"),"end-date":date("2008-07-25")}]}
+{"id":10283941,"id-copy":10283941,"alias":"Jeffie","name":"JeffieChappel","user-since":datetime("2012-06-17T10:07:53"),"user-since-copy":datetime("2012-06-17T10:07:53"),"friend-ids":{{37665650,44995551,8518132,25975224,22980129,41720034,42152946,26671472,25698917,24270208,36866555,6728174,46967331,31563323,1382901,6764335,35373496}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2005-01-26")}]}
+{"id":9877837,"id-copy":9877837,"alias":"Marilee","name":"MarileeDowning","user-since":datetime("2007-09-06T15:02:25"),"user-since-copy":datetime("2007-09-06T15:02:25"),"friend-ids":{{3032720,7000379,16658012,33487490,624779,13480315,8308906,6949934,9472007,36568244,41737195,1310478,42870077,46663613}},"employment":[{"organization-name":"Lexicone","start-date":date("2005-10-09")}]}
+{"id":11533327,"id-copy":11533327,"alias":"Miguel","name":"MiguelSteiner","user-since":datetime("2007-12-08T18:21:30"),"user-since-copy":datetime("2007-12-08T18:21:30"),"friend-ids":{{41619494,4881397,29302201,26654760,9690024,15599321,37163728,2420315,46258007,15076674,6757461}},"employment":[{"organization-name":"Streettax","start-date":date("2001-08-19"),"end-date":date("2008-10-15")}]}
+{"id":11721010,"id-copy":11721010,"alias":"Eliot","name":"EliotTennant","user-since":datetime("2009-07-25T22:16:20"),"user-since-copy":datetime("2009-07-25T22:16:20"),"friend-ids":{{41972338,13293762,47012929,13695904,25235210,39246961,36832468,26854695,3046764,17117110,10902219,36959080,32665222}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-11-26")}]}
+{"id":10346116,"id-copy":10346116,"alias":"Breana","name":"BreanaPainter","user-since":datetime("2012-04-05T12:15:17"),"user-since-copy":datetime("2012-04-05T12:15:17"),"friend-ids":{{39999376,5382299,36254541,16829210,7084172,13545656,24681698,34171417,28514693,8090159,35046661,44544921,47754565,28732689,19680056,21398367,39260450}},"employment":[{"organization-name":"Redelectronics","start-date":date("2012-08-24"),"end-date":date("2012-08-24")}]}
+{"id":10804771,"id-copy":10804771,"alias":"Delicia","name":"DeliciaPittman","user-since":datetime("2008-04-12T01:07:13"),"user-since-copy":datetime("2008-04-12T01:07:13"),"friend-ids":{{35228090}},"employment":[{"organization-name":"Freshfix","start-date":date("2006-08-16")}]}
+{"id":9041443,"id-copy":9041443,"alias":"Maria","name":"MariaWard","user-since":datetime("2006-12-25T01:24:40"),"user-since-copy":datetime("2006-12-25T01:24:40"),"friend-ids":{{10660010,19103672,11300656,44383404,36523093,11434370,34405687,30889551,4843181,22025114,26395363,8607483,25294309}},"employment":[{"organization-name":"Qvohouse","start-date":date("2005-06-13")}]}
+{"id":9505936,"id-copy":9505936,"alias":"Kerrie","name":"KerrieGadow","user-since":datetime("2005-06-26T08:47:14"),"user-since-copy":datetime("2005-06-26T08:47:14"),"friend-ids":{{46457424,17421010,11336465,19785227}},"employment":[{"organization-name":"over-it","start-date":date("2004-12-08"),"end-date":date("2010-04-11")}]}
+{"id":11989645,"id-copy":11989645,"alias":"Weston","name":"WestonPershing","user-since":datetime("2010-04-02T17:25:31"),"user-since-copy":datetime("2010-04-02T17:25:31"),"friend-ids":{{11689127}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2006-03-27")}]}
+{"id":11250445,"id-copy":11250445,"alias":"Charlie","name":"CharlieHaynes","user-since":datetime("2009-06-08T22:50:05"),"user-since-copy":datetime("2009-06-08T22:50:05"),"friend-ids":{{18548568,33185990,25924893,44738376,17285644,30895698,40664753,45663520,13757940,46543434,27472319,7112791,45257808,29363383,24726693,39990597,36277676,6623887,42795972,29019649,22035134,1362080,9071131}},"employment":[{"organization-name":"Quadlane","start-date":date("2008-02-21"),"end-date":date("2009-12-28")}]}
+{"id":11276305,"id-copy":11276305,"alias":"Salome","name":"SalomeGongaware","user-since":datetime("2007-06-05T10:15:14"),"user-since-copy":datetime("2007-06-05T10:15:14"),"friend-ids":{{17354378,35576200,42905756,44408264,45572153,18424890,39234162,42837501,38464194,45237502,30396078,16316605,32231800,35417394,32796520,13885091,31520983,4624403,18144193,45707906,8211336,2864876}},"employment":[{"organization-name":"Scotcity","start-date":date("2002-03-16")}]}
+{"id":9974485,"id-copy":9974485,"alias":"Leo","name":"LeoRawls","user-since":datetime("2005-02-12T12:01:58"),"user-since-copy":datetime("2005-02-12T12:01:58"),"friend-ids":{{41189338,33744557,2485502,8308490,43237410}},"employment":[{"organization-name":"itlab","start-date":date("2002-11-05"),"end-date":date("2009-04-12")}]}
+{"id":10123051,"id-copy":10123051,"alias":"Rowland","name":"RowlandWaldron","user-since":datetime("2011-08-01T17:20:14"),"user-since-copy":datetime("2011-08-01T17:20:14"),"friend-ids":{{7693849,5416143,10885197,39771258,41278769,16236783,18739058,2293485,32013369,34882536,14339467,3680575,4461977,33715303,26345760,45729149,17585375,39496021}},"employment":[{"organization-name":"Codetechno","start-date":date("2001-12-10"),"end-date":date("2006-04-07")}]}
+{"id":9098314,"id-copy":9098314,"alias":"Terrance","name":"TerranceWilkerson","user-since":datetime("2010-07-01T06:01:32"),"user-since-copy":datetime("2010-07-01T06:01:32"),"friend-ids":{{32477103,38306013,36022406,25594192,10966661,28537611,5444323,16012053,43228208,30344050,22600011,42820310,37103995,6359985}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2012-01-24")}]}
+{"id":9555157,"id-copy":9555157,"alias":"Alea","name":"AleaWallick","user-since":datetime("2009-11-12T19:32:16"),"user-since-copy":datetime("2009-11-12T19:32:16"),"friend-ids":{{9936033,18972695,22198051,44425768,37636218,25373418,17204473,6543589,23627204,40204583,18664982,27647616,43332268,41812682}},"employment":[{"organization-name":"Ganjatax","start-date":date("2009-02-17")}]}
+{"id":10837876,"id-copy":10837876,"alias":"Tianna","name":"TiannaOppenheimer","user-since":datetime("2006-05-14T01:19:23"),"user-since-copy":datetime("2006-05-14T01:19:23"),"friend-ids":{{8389212,20540523,37708985,22298925,5938365,34705514,39174355,44283530,44597508,37912034,45434053,47086440,6559664,12451920,47639456,39030619,24239344,2566247,27102794}},"employment":[{"organization-name":"Rungozoom","start-date":date("2008-04-03")}]}
+{"id":9467614,"id-copy":9467614,"alias":"Eloisa","name":"EloisaEvans","user-since":datetime("2012-01-20T01:00:51"),"user-since-copy":datetime("2012-01-20T01:00:51"),"friend-ids":{{}},"employment":[{"organization-name":"Alphadax","start-date":date("2000-11-03"),"end-date":date("2003-01-14")}]}
+{"id":9951325,"id-copy":9951325,"alias":"Sarah","name":"SarahRockwell","user-since":datetime("2009-08-25T01:56:51"),"user-since-copy":datetime("2009-08-25T01:56:51"),"friend-ids":{{14846488,32939876,43509116,36687501,6496360,47346160,20558288,21828060}},"employment":[{"organization-name":"Quoline","start-date":date("2002-11-18")}]}
+{"id":9596080,"id-copy":9596080,"alias":"Yolonda","name":"YolondaUlery","user-since":datetime("2012-03-02T19:57:32"),"user-since-copy":datetime("2012-03-02T19:57:32"),"friend-ids":{{22382589,22012001,13142890,44320162,10358341,14975,43101433,10324321,14791134,25984312,11075173,44140537,40528755,27384004,40022140,10650900,37789740,6928495,22130557,47679224,40973393,37190617,35395183}},"employment":[{"organization-name":"Freshfix","start-date":date("2012-03-03")}]}
+{"id":10624381,"id-copy":10624381,"alias":"Ryana","name":"RyanaKimmons","user-since":datetime("2007-09-04T15:42:08"),"user-since-copy":datetime("2007-09-04T15:42:08"),"friend-ids":{{36219003,5135252,24653726,4767631,21595268,4154414,31857818,9711256,20793102,14509650}},"employment":[{"organization-name":"Plexlane","start-date":date("2006-06-21")}]}
+{"id":11965318,"id-copy":11965318,"alias":"Donella","name":"DonellaPriebe","user-since":datetime("2010-10-25T19:45:41"),"user-since-copy":datetime("2010-10-25T19:45:41"),"friend-ids":{{40521325}},"employment":[{"organization-name":"Doublezone","start-date":date("2010-11-19"),"end-date":date("2011-08-18")}]}
+{"id":11135899,"id-copy":11135899,"alias":"Bailey","name":"BaileyMoonshower","user-since":datetime("2011-08-28T07:36:28"),"user-since-copy":datetime("2011-08-28T07:36:28"),"friend-ids":{{29802790,16418079}},"employment":[{"organization-name":"goldendexon","start-date":date("2010-05-17")}]}
+{"id":10100707,"id-copy":10100707,"alias":"Brittni","name":"BrittniEaster","user-since":datetime("2008-10-03T02:27:48"),"user-since-copy":datetime("2008-10-03T02:27:48"),"friend-ids":{{28725707,8497950,18892135,1016149,32023719,34079976,39582966,15469248,14059091,6681733,18398487,41385960}},"employment":[{"organization-name":"Zuncan","start-date":date("2006-04-21")}]}
+{"id":9774613,"id-copy":9774613,"alias":"Kaycee","name":"KayceeGeyer","user-since":datetime("2008-12-19T06:09:36"),"user-since-copy":datetime("2008-12-19T06:09:36"),"friend-ids":{{35485847,33668074,21309976,40428525,40450508,30804358,1365381,5197688,37844952,4076960,28446817,20696590,23896488,33454126,21411087,9300550,12986775,36731809,47850175,9503217,22481614,29556396,15013896,14407126}},"employment":[{"organization-name":"Lexicone","start-date":date("2001-03-23"),"end-date":date("2003-01-16")}]}
+{"id":10487029,"id-copy":10487029,"alias":"Fredericka","name":"FrederickaShea","user-since":datetime("2011-04-07T06:12:40"),"user-since-copy":datetime("2011-04-07T06:12:40"),"friend-ids":{{45223639,1019151,30626857,10247171,36952244,36646177,2396690,26604216,19215860,20900949,14160764}},"employment":[{"organization-name":"Ganjatax","start-date":date("2012-01-08")}]}
+{"id":11506045,"id-copy":11506045,"alias":"Marci","name":"MarciSaltser","user-since":datetime("2011-08-05T00:36:14"),"user-since-copy":datetime("2011-08-05T00:36:14"),"friend-ids":{{44810951,11599851,4960763,13454104,22872317,44594135,15792938}},"employment":[{"organization-name":"Y-geohex","start-date":date("2002-06-22"),"end-date":date("2009-08-20")}]}
+{"id":11246161,"id-copy":11246161,"alias":"Jemima","name":"JemimaJube","user-since":datetime("2009-10-13T13:44:48"),"user-since-copy":datetime("2009-10-13T13:44:48"),"friend-ids":{{35264732,26686176,37947249,9511009,20544975,21318354,2417039,15051823,23702057,34446389,15435804,42646090,14791709}},"employment":[{"organization-name":"Solfix","start-date":date("2012-02-26")}]}
+{"id":10458316,"id-copy":10458316,"alias":"Nivek","name":"NivekHarper","user-since":datetime("2009-06-27T16:14:07"),"user-since-copy":datetime("2009-06-27T16:14:07"),"friend-ids":{{28377255,40295259,41434117,37075748,12913111,1533923,393103,31161713,13106373,924904,14927212,7552938,8299772,28404911,45464821,34440085,36216015,2915789,13470222,34755494,29250423}},"employment":[{"organization-name":"Basecone","start-date":date("2002-03-28"),"end-date":date("2010-12-09")}]}
+{"id":11131756,"id-copy":11131756,"alias":"Sharlene","name":"SharleneFinlay","user-since":datetime("2006-01-11T00:34:50"),"user-since-copy":datetime("2006-01-11T00:34:50"),"friend-ids":{{47024803,17225785,29871165,14503159,22992924,38939801,44563447,101625,40957129,24838380,7187619,45283524,31617405,517806,28714183,32966332,24006006}},"employment":[{"organization-name":"Medflex","start-date":date("2008-02-16"),"end-date":date("2011-09-12")}]}
+{"id":11140483,"id-copy":11140483,"alias":"Nena","name":"NenaBullard","user-since":datetime("2008-02-23T10:24:08"),"user-since-copy":datetime("2008-02-23T10:24:08"),"friend-ids":{{26438400,45201681,12155417,43414633,14267296,40906639,8768744,46840439,43848021,24521652,41247005,44999926,13062334,47731182}},"employment":[{"organization-name":"Sancone","start-date":date("2001-05-16")}]}
+{"id":10017829,"id-copy":10017829,"alias":"Adam","name":"AdamTrovato","user-since":datetime("2009-04-15T20:21:48"),"user-since-copy":datetime("2009-04-15T20:21:48"),"friend-ids":{{7572792,20961281,47727918,25262896,33740076,14418354,42807653,34174665,12459426,28777106,44409513,39753872,9172361,36746114,196755}},"employment":[{"organization-name":"Doncare","start-date":date("2007-09-25")}]}
+{"id":10238749,"id-copy":10238749,"alias":"Elspeth","name":"ElspethFilby","user-since":datetime("2010-02-08T22:55:13"),"user-since-copy":datetime("2010-02-08T22:55:13"),"friend-ids":{{307224,16533888}},"employment":[{"organization-name":"goldendexon","start-date":date("2011-12-10")}]}
+{"id":11155816,"id-copy":11155816,"alias":"Titty","name":"TittyOneal","user-since":datetime("2009-06-01T06:21:44"),"user-since-copy":datetime("2009-06-01T06:21:44"),"friend-ids":{{37016026,32220220,47720886,10358045,7678433,22148913,18800507,17043803,29852152,11426875,44761613,32002053,14686180,26744098,34991446,38818677,24977770}},"employment":[{"organization-name":"Zuncan","start-date":date("2012-05-11"),"end-date":date("2012-05-08")}]}
+{"id":9562348,"id-copy":9562348,"alias":"Jefferson","name":"JeffersonKeister","user-since":datetime("2005-06-11T01:42:58"),"user-since-copy":datetime("2005-06-11T01:42:58"),"friend-ids":{{43801762}},"employment":[{"organization-name":"strongex","start-date":date("2005-07-26"),"end-date":date("2011-12-02")}]}
+{"id":10515721,"id-copy":10515721,"alias":"Mariano","name":"MarianoTrout","user-since":datetime("2007-08-27T09:33:28"),"user-since-copy":datetime("2007-08-27T09:33:28"),"friend-ids":{{18516004,4847094,31548989,28302698,18308169,15068883,33358074,19739053,34017693}},"employment":[{"organization-name":"tresline","start-date":date("2004-10-18")}]}
+{"id":10148251,"id-copy":10148251,"alias":"Ghislaine","name":"GhislaineFowler","user-since":datetime("2005-12-08T05:25:56"),"user-since-copy":datetime("2005-12-08T05:25:56"),"friend-ids":{{14692731,29964772}},"employment":[{"organization-name":"physcane","start-date":date("2008-12-27"),"end-date":date("2008-04-02")}]}
+{"id":10086913,"id-copy":10086913,"alias":"Margaretta","name":"MargarettaPfeifer","user-since":datetime("2012-03-04T14:47:18"),"user-since-copy":datetime("2012-03-04T14:47:18"),"friend-ids":{{9800482,3761286,34428154,18082184,14845214,33053674,46786785,22235473,23677556,24819784,47587008,36939436,14987278}},"employment":[{"organization-name":"Zununoing","start-date":date("2003-07-08"),"end-date":date("2010-03-01")}]}
+{"id":10601758,"id-copy":10601758,"alias":"Blossom","name":"BlossomClark","user-since":datetime("2011-08-16T23:44:16"),"user-since-copy":datetime("2011-08-16T23:44:16"),"friend-ids":{{22624576,6945784,47816004,8072206,23953052,22668193,8668574,2269602,39137309,38996903,23516086,31166264,28322741,46296094,36547681,7287738,15727604,13556387,2624138}},"employment":[{"organization-name":"Ontotanin","start-date":date("2004-10-15"),"end-date":date("2008-07-17")}]}
+{"id":11729626,"id-copy":11729626,"alias":"Kassandra","name":"KassandraBaker","user-since":datetime("2010-12-26T12:18:49"),"user-since-copy":datetime("2010-12-26T12:18:49"),"friend-ids":{{2336026,15350108,46098823,35193308,34644345,45989141,31179029,15991657,12863616,18297246,26571280,16935684,31339122,10623785,24666322,23094237,28117245,40096052,37538843,8085609,2437482,8885815,42016898,4654048}},"employment":[{"organization-name":"Transhigh","start-date":date("2007-07-10")}]}
+{"id":9859726,"id-copy":9859726,"alias":"Taryn","name":"TarynGisiko","user-since":datetime("2010-12-28T21:42:56"),"user-since-copy":datetime("2010-12-28T21:42:56"),"friend-ids":{{45036313,47860435,40658528,4106429,25411752,7216290,20549107,28317961,43600081,6359672,36131464,19078372,4379305,884797,11605059,6467240,23316141}},"employment":[{"organization-name":"Techitechi","start-date":date("2007-06-26"),"end-date":date("2010-08-04")}]}
+{"id":11788096,"id-copy":11788096,"alias":"Camie","name":"CamieCressman","user-since":datetime("2007-10-25T23:38:14"),"user-since-copy":datetime("2007-10-25T23:38:14"),"friend-ids":{{29310801,37328820,47367940,36796774,21244245,7126676,8254586,47578674,39514952,33623672,12854915,6679164,44128364,44434013,20530444,12243267}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2000-06-20")}]}
+{"id":11809528,"id-copy":11809528,"alias":"Donya","name":"DonyaNash","user-since":datetime("2008-06-09T09:42:48"),"user-since-copy":datetime("2008-06-09T09:42:48"),"friend-ids":{{25365000,20270987,39083310,16364767,1960249,39747742,17169019,780802,37012712,27956954,35502958,10600365,38247667,47815777,25182855,13670701,27795853,24952265}},"employment":[{"organization-name":"Goldcity","start-date":date("2011-10-15")}]}
+{"id":10768810,"id-copy":10768810,"alias":"Gaston","name":"GastonBender","user-since":datetime("2008-05-24T17:27:14"),"user-since-copy":datetime("2008-05-24T17:27:14"),"friend-ids":{{29652235,40180625,34608178,43814186,9682855,24692412,33119254,20480079,35147289,24629496,1449575}},"employment":[{"organization-name":"Solophase","start-date":date("2010-04-06")}]}
+{"id":10280533,"id-copy":10280533,"alias":"Normand","name":"NormandAckerley","user-since":datetime("2008-05-18T00:44:35"),"user-since-copy":datetime("2008-05-18T00:44:35"),"friend-ids":{{46908522,2002203,15632192,3790633,21300428,15452344,34478785,18864214,32842683,10486268,2496859}},"employment":[{"organization-name":"Roundhex","start-date":date("2010-12-07")}]}
+{"id":10661566,"id-copy":10661566,"alias":"Cathy","name":"CathyKight","user-since":datetime("2007-07-17T18:53:31"),"user-since-copy":datetime("2007-07-17T18:53:31"),"friend-ids":{{19477294,31919442,6947933,16858850,21921187,21214480,19616226,2133662,42362248,7534944,12953803,41148200,30043772,38130157,36623612,45371575,25019205,10260656}},"employment":[{"organization-name":"Voltbam","start-date":date("2008-12-09"),"end-date":date("2008-01-04")}]}
+{"id":11489143,"id-copy":11489143,"alias":"Clover","name":"CloverWest","user-since":datetime("2012-04-14T13:56:22"),"user-since-copy":datetime("2012-04-14T13:56:22"),"friend-ids":{{14606516,25835971,10555192,4853088,43631398,45670230,43866490,25690294,22040370,7047997,3374421,34831455,31517002,2998558,40893307,40067725,1601716,43041725,8953042,33848939}},"employment":[{"organization-name":"Xx-drill","start-date":date("2005-02-03"),"end-date":date("2006-06-26")}]}
+{"id":11678242,"id-copy":11678242,"alias":"Andy","name":"AndyPritchard","user-since":datetime("2008-05-26T06:52:12"),"user-since-copy":datetime("2008-05-26T06:52:12"),"friend-ids":{{24351029,7396495,11653891,24314059,17256129,19177689,23024021,15135862,9201238,24204194}},"employment":[{"organization-name":"Sublamdox","start-date":date("2004-02-06"),"end-date":date("2011-10-22")}]}
+{"id":11762239,"id-copy":11762239,"alias":"Guillermo","name":"GuillermoCallison","user-since":datetime("2009-02-12T13:46:40"),"user-since-copy":datetime("2009-02-12T13:46:40"),"friend-ids":{{3494924,650832,22099424,11629223,45581083,206762,27794516,7639789,31794781,22985617,17273963,9120417,9496942,47474589,47872578,34639130,37695869,41346670,7789418,24870369,31562430,2414862,41928569}},"employment":[{"organization-name":"freshdox","start-date":date("2008-07-26")}]}
+{"id":11804755,"id-copy":11804755,"alias":"Humbert","name":"HumbertArmitage","user-since":datetime("2008-01-01T21:14:34"),"user-since-copy":datetime("2008-01-01T21:14:34"),"friend-ids":{{15498777,1984479,18672418,13137212,17931875,10446256,39250716,9422828,35469173,35940705,44217206}},"employment":[{"organization-name":"Voltlane","start-date":date("2005-11-12")}]}
+{"id":9802330,"id-copy":9802330,"alias":"Kirby","name":"KirbyKnopsnider","user-since":datetime("2011-12-18T01:10:12"),"user-since-copy":datetime("2011-12-18T01:10:12"),"friend-ids":{{3703876,46564552,9263120,39930137,36202804,45164241,7778394,2527495,2831079,33834588,42759211,2766215,36344152,5218620,1190357,30615313,25434877,43958817,23617510}},"employment":[{"organization-name":"U-electrics","start-date":date("2008-02-01")}]}
+{"id":11515828,"id-copy":11515828,"alias":"Christa","name":"ChristaWain","user-since":datetime("2007-05-01T13:32:18"),"user-since-copy":datetime("2007-05-01T13:32:18"),"friend-ids":{{9081871,27897837,47641133,1224070,41007475,39553691,10757036,28663201,44842180,24894191,42128523,30703082,27281648,9786943}},"employment":[{"organization-name":"itlab","start-date":date("2012-05-04")}]}
+{"id":9497698,"id-copy":9497698,"alias":"Jenny","name":"JennyBiery","user-since":datetime("2007-07-24T17:20:06"),"user-since-copy":datetime("2007-07-24T17:20:06"),"friend-ids":{{37832227,17148339,38184683,45775690,17511050,1866913,30631091,5996302,3796747,33135567,5930972,9509054,44003369,34299276,16135297,15435466,42464299,34961792,47264306,30734198,26192613}},"employment":[{"organization-name":"Plexlane","start-date":date("2007-10-02"),"end-date":date("2011-09-20")}]}
+{"id":9005248,"id-copy":9005248,"alias":"Jervis","name":"JervisWarrick","user-since":datetime("2007-02-06T17:54:17"),"user-since-copy":datetime("2007-02-06T17:54:17"),"friend-ids":{{5038062,15101135,28136073,10706469,8706391,10623870,1759405,37020186,17173998,14985805,19308437,43696985,46650868,25621415,14252531,44491166,42536769,33614525,34665072,640793}},"employment":[{"organization-name":"Icerunin","start-date":date("2010-08-21")}]}
+{"id":9784687,"id-copy":9784687,"alias":"Larrie","name":"LarrieStroh","user-since":datetime("2005-12-03T13:45:30"),"user-since-copy":datetime("2005-12-03T13:45:30"),"friend-ids":{{38055237,43436653,21194063,30405058,7754813,14616686,3434657,24778389,5653770,8600235,44560871,4379727,32140404,35445864,24133933,21379278,45626842,25710375,25970333,16831917}},"employment":[{"organization-name":"Groovetex","start-date":date("2006-09-18")}]}
+{"id":10712731,"id-copy":10712731,"alias":"Abigail","name":"AbigailKunkle","user-since":datetime("2011-07-20T07:10:43"),"user-since-copy":datetime("2011-07-20T07:10:43"),"friend-ids":{{35920648,38798778,17160209,16674423,44247736,45731986,29605307,123608,46926535,41274265,36397206,16900492,19895463,10043680,42549381,21006240,13037274,25867242,34428167,953419,2284340,32321044,2351589,30797666}},"employment":[{"organization-name":"over-it","start-date":date("2002-08-11"),"end-date":date("2002-12-01")}]}
+{"id":11562148,"id-copy":11562148,"alias":"Rexana","name":"RexanaStange","user-since":datetime("2012-08-13T20:11:05"),"user-since-copy":datetime("2012-08-13T20:11:05"),"friend-ids":{{22418981,44892347,43890424,38530948,33178064}},"employment":[{"organization-name":"physcane","start-date":date("2004-11-21"),"end-date":date("2007-11-01")}]}
+{"id":9020338,"id-copy":9020338,"alias":"Shenika","name":"ShenikaColdsmith","user-since":datetime("2011-02-22T08:03:05"),"user-since-copy":datetime("2011-02-22T08:03:05"),"friend-ids":{{28029790,45719398,12088661,4134025,27354070,46504723,23155578,3370020,26477155,27314367,7672726,41117417}},"employment":[{"organization-name":"Codetechno","start-date":date("2011-04-18")}]}
+{"id":10771030,"id-copy":10771030,"alias":"Jen","name":"JenZaun","user-since":datetime("2006-12-02T14:42:43"),"user-since-copy":datetime("2006-12-02T14:42:43"),"friend-ids":{{38166077}},"employment":[{"organization-name":"Quoline","start-date":date("2003-12-01"),"end-date":date("2010-04-12")}]}
+{"id":11786815,"id-copy":11786815,"alias":"Micheal","name":"MichealTreeby","user-since":datetime("2008-06-04T14:59:23"),"user-since-copy":datetime("2008-06-04T14:59:23"),"friend-ids":{{15590922,1367468,37464776,21877607,38646966,46702919,46771039,4688915,41827211,6556380}},"employment":[{"organization-name":"Villa-dox","start-date":date("2003-09-17")}]}
+{"id":10953628,"id-copy":10953628,"alias":"Clement","name":"ClementHoenshell","user-since":datetime("2009-01-24T03:52:54"),"user-since-copy":datetime("2009-01-24T03:52:54"),"friend-ids":{{24684431,16961296,13566818}},"employment":[{"organization-name":"highfax","start-date":date("2011-05-07")}]}
+{"id":10786438,"id-copy":10786438,"alias":"Sherika","name":"SherikaShick","user-since":datetime("2005-05-18T21:46:18"),"user-since-copy":datetime("2005-05-18T21:46:18"),"friend-ids":{{11188876,12936787,43459190,40396919,7166644,20299758}},"employment":[{"organization-name":"Technohow","start-date":date("2002-06-09")}]}
+{"id":10650265,"id-copy":10650265,"alias":"Kristia","name":"KristiaCowart","user-since":datetime("2005-09-27T20:13:12"),"user-since-copy":datetime("2005-09-27T20:13:12"),"friend-ids":{{41553475,45442923,20846576,6432869,40830841}},"employment":[{"organization-name":"Kanelectrics","start-date":date("2006-05-05")}]}
+{"id":9050164,"id-copy":9050164,"alias":"Haydee","name":"HaydeeCook","user-since":datetime("2005-08-28T12:13:59"),"user-since-copy":datetime("2005-08-28T12:13:59"),"friend-ids":{{26484166,27686644,42277018,5893537,34617524,12158738,41566344,30653024,23636324,24072660,1784294,38620941,40846838,30303402,27004887,35907658,42893556,10118575,47861482}},"employment":[{"organization-name":"Newfase","start-date":date("2007-11-23")}]}
+{"id":11538001,"id-copy":11538001,"alias":"Milo","name":"MiloGarland","user-since":datetime("2007-09-12T09:40:42"),"user-since-copy":datetime("2007-09-12T09:40:42"),"friend-ids":{{7363153,7252759}},"employment":[{"organization-name":"Newhotplus","start-date":date("2011-09-03"),"end-date":date("2011-10-27")}]}
+{"id":11162977,"id-copy":11162977,"alias":"Orson","name":"OrsonFlick","user-since":datetime("2010-02-17T21:05:53"),"user-since-copy":datetime("2010-02-17T21:05:53"),"friend-ids":{{12213318,19062680,20035734,5154338,24649936,30379574,38611249,36143038,13393939,14976281,34963200,4510968,45722224,18820241}},"employment":[{"organization-name":"Strongtone","start-date":date("2001-03-14"),"end-date":date("2001-10-15")}]}
+{"id":11152162,"id-copy":11152162,"alias":"Tennille","name":"TennilleGongaware","user-since":datetime("2008-12-22T17:22:19"),"user-since-copy":datetime("2008-12-22T17:22:19"),"friend-ids":{{38167013,48016045,45757020,26256748,14740496,36818162,43284365,29637839,30820213,535748,31611626}},"employment":[{"organization-name":"Newcom","start-date":date("2005-07-27")}]}
+{"id":10453144,"id-copy":10453144,"alias":"Jason","name":"JasonSachse","user-since":datetime("2009-01-25T10:27:17"),"user-since-copy":datetime("2009-01-25T10:27:17"),"friend-ids":{{12949882,32048809,23087453,3994051,20775019,22184704,38106058,34520240,13724092,16309751,25955640,4812195,40546554,12695295,16574455,38615670,43405164,7997407,12239790}},"employment":[{"organization-name":"tresline","start-date":date("2005-08-01"),"end-date":date("2008-02-08")}]}
+{"id":9574393,"id-copy":9574393,"alias":"Ghislaine","name":"GhislaineTaylor","user-since":datetime("2005-01-23T07:49:26"),"user-since-copy":datetime("2005-01-23T07:49:26"),"friend-ids":{{23799181,25411427,3758740,47542325,41070945,45261892,23309481}},"employment":[{"organization-name":"tresline","start-date":date("2003-04-15")}]}
+{"id":10635319,"id-copy":10635319,"alias":"Rusty","name":"RustyStange","user-since":datetime("2010-08-17T17:30:37"),"user-since-copy":datetime("2010-08-17T17:30:37"),"friend-ids":{{28180565,25608756}},"employment":[{"organization-name":"Icerunin","start-date":date("2006-07-07")}]}
+{"id":9366253,"id-copy":9366253,"alias":"Emma","name":"EmmaKnisely","user-since":datetime("2012-07-08T20:39:00"),"user-since-copy":datetime("2012-07-08T20:39:00"),"friend-ids":{{40874500,35049897,29559982,42737582,11405173,38919458,26268603,38582942,13758558,16949073}},"employment":[{"organization-name":"Ganjatax","start-date":date("2009-11-18")}]}
+{"id":11175613,"id-copy":11175613,"alias":"Cuthbert","name":"CuthbertHoover","user-since":datetime("2008-04-25T01:12:49"),"user-since-copy":datetime("2008-04-25T01:12:49"),"friend-ids":{{27333562,43896730,6549030,19576014,4728367,15430069,22146931,44593208,14070342,27801009,6735368,35798322,47213791,2388166}},"employment":[{"organization-name":"Codetechno","start-date":date("2004-07-18")}]}
+{"id":11758474,"id-copy":11758474,"alias":"Xavier","name":"XavierAtweeke","user-since":datetime("2011-10-03T12:35:37"),"user-since-copy":datetime("2011-10-03T12:35:37"),"friend-ids":{{30110740,41016650,23732518,14585316,34474077,47591093,10803514,8912354,43455040,21960801,31978150,40693811,14585416,36411476,20556412,44978412,7266670,506620,7686872}},"employment":[{"organization-name":"Quoline","start-date":date("2004-03-07")}]}
+{"id":11540278,"id-copy":11540278,"alias":"Flora","name":"FloraSaltser","user-since":datetime("2007-11-20T08:52:26"),"user-since-copy":datetime("2007-11-20T08:52:26"),"friend-ids":{{44172124,43836609,2821020,356092,25456578,14806637,19970466,15369859,23267393,34480680,42574031,39606777,17221367,19617483,1364901,21402012,4999365,31098654,34512618,44652673,14757091,9755310,39190510}},"employment":[{"organization-name":"strongex","start-date":date("2012-07-07")}]}
+{"id":11373598,"id-copy":11373598,"alias":"Dina","name":"DinaDriggers","user-since":datetime("2010-01-06T22:56:18"),"user-since-copy":datetime("2010-01-06T22:56:18"),"friend-ids":{{8839886,10146989,10877857,11710726,5699142,27984085,12834284}},"employment":[{"organization-name":"U-ron","start-date":date("2012-07-25")}]}
+{"id":10322023,"id-copy":10322023,"alias":"Shanita","name":"ShanitaBeedell","user-since":datetime("2011-06-09T23:50:09"),"user-since-copy":datetime("2011-06-09T23:50:09"),"friend-ids":{{22628842,2169935,20656034,9086684,17234788,11936164,12465122,2543006,40067557,36767662,633930,41805132,13246529,43801547,44953975,36902947,34935791,22923033,28190533,18230134,9484458,21184932}},"employment":[{"organization-name":"Groovetex","start-date":date("2011-10-10")}]}
+{"id":10659022,"id-copy":10659022,"alias":"Cecelia","name":"CeceliaHandyside","user-since":datetime("2007-02-22T12:42:30"),"user-since-copy":datetime("2007-02-22T12:42:30"),"friend-ids":{{9051,38746030,6178049,22068473,25755202,11577837,28994476}},"employment":[{"organization-name":"Ronholdings","start-date":date("2004-07-09"),"end-date":date("2009-10-14")}]}
+{"id":10400386,"id-copy":10400386,"alias":"Marion","name":"MarionBuck","user-since":datetime("2006-06-22T03:35:25"),"user-since-copy":datetime("2006-06-22T03:35:25"),"friend-ids":{{35854700,8766966,41860546,25745457,12225165,15412904,39841282,5879215,24965438,4636142,43652954,36414405,34931848,38550959,30395999,44263220,8167212,35555246,11177002,29078503}},"employment":[{"organization-name":"Hatcom","start-date":date("2000-08-28")}]}
+{"id":9599647,"id-copy":9599647,"alias":"Alexandria","name":"AlexandriaWade","user-since":datetime("2012-06-25T06:48:48"),"user-since-copy":datetime("2012-06-25T06:48:48"),"friend-ids":{{20910866,20843338,8182424,21070448,43548111,39370893,26760127,11135506}},"employment":[{"organization-name":"subtam","start-date":date("2011-06-02")}]}
+{"id":9962236,"id-copy":9962236,"alias":"Craig","name":"CraigKight","user-since":datetime("2010-02-15T15:58:03"),"user-since-copy":datetime("2010-02-15T15:58:03"),"friend-ids":{{45604304,40911167,39517053,6912584,898627,8412812,33530827,30135549,14762146,46313211,21143796,39820220,11462372,23575315}},"employment":[{"organization-name":"Coneflex","start-date":date("2001-02-05"),"end-date":date("2008-01-04")}]}
+{"id":11404780,"id-copy":11404780,"alias":"Carol","name":"CarolCox","user-since":datetime("2009-07-07T23:58:07"),"user-since-copy":datetime("2009-07-07T23:58:07"),"friend-ids":{{41450896,12332484,18515318,39039576,2336271,47313837,4655597,40110200,7357446,24291515,8898678,28911118,20372890,1296082,42558011,5719716,6830197}},"employment":[{"organization-name":"Lexitechno","start-date":date("2005-01-14")}]}
+{"id":11016043,"id-copy":11016043,"alias":"Ellis","name":"EllisVorrasi","user-since":datetime("2009-08-26T16:43:17"),"user-since-copy":datetime("2009-08-26T16:43:17"),"friend-ids":{{41000811,12639978,14487796,39651858,40189282,7834125,44416511,28673665}},"employment":[{"organization-name":"overtech","start-date":date("2008-01-21"),"end-date":date("2008-04-26")}]}
+{"id":11090788,"id-copy":11090788,"alias":"Randy","name":"RandyClose","user-since":datetime("2005-07-26T19:29:20"),"user-since-copy":datetime("2005-07-26T19:29:20"),"friend-ids":{{43392502,7581874,13279708,16989391,32340594,7048512,33084049,16279611,21735714,23485799,18185370,43945382,41653020,13517043,35395274,24133848,15355027,4752815,15007500,25733540,2114558,37909789,2805493,16521087}},"employment":[{"organization-name":"Voltlane","start-date":date("2004-09-14")}]}
+{"id":10529809,"id-copy":10529809,"alias":"Aric","name":"AricLauffer","user-since":datetime("2007-05-18T09:08:29"),"user-since-copy":datetime("2007-05-18T09:08:29"),"friend-ids":{{36647795,13183862,5313167,36450019,46412788,47789981,4012027,35872968,3903895}},"employment":[{"organization-name":"geomedia","start-date":date("2011-09-22")}]}
+{"id":11426248,"id-copy":11426248,"alias":"Chryssa","name":"ChryssaHincken","user-since":datetime("2005-06-16T01:11:36"),"user-since-copy":datetime("2005-06-16T01:11:36"),"friend-ids":{{47119545}},"employment":[{"organization-name":"Sublamdox","start-date":date("2003-11-20"),"end-date":date("2003-10-07")}]}
+{"id":10882393,"id-copy":10882393,"alias":"Erica","name":"EricaHynes","user-since":datetime("2006-09-16T16:39:05"),"user-since-copy":datetime("2006-09-16T16:39:05"),"friend-ids":{{23491370,13390922,19685128,47763240,9493285,10823383,45076071,14858340,12545499,40367152,2150593,45723007,21362425,25435409,776198,8016739,21691528,21036410,3131225,20078710,28405287,15599245,39126345,36208574}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2012-05-14"),"end-date":date("2012-05-22")}]}
+{"id":10188805,"id-copy":10188805,"alias":"Margarita","name":"MargaritaBrinigh","user-since":datetime("2011-06-26T06:22:38"),"user-since-copy":datetime("2011-06-26T06:22:38"),"friend-ids":{{39275311,42262790,35041935,12137373,8507536}},"employment":[{"organization-name":"Groovetex","start-date":date("2004-03-17")}]}
+{"id":11551078,"id-copy":11551078,"alias":"Percy","name":"PercyStocker","user-since":datetime("2012-01-12T15:14:02"),"user-since-copy":datetime("2012-01-12T15:14:02"),"friend-ids":{{8927010,25565873,1309019,9736505,27953053,6619625,45562540,32022492,1535156,11343220,40057278,5452463,36005348,35072612,31954888}},"employment":[{"organization-name":"Sumlane","start-date":date("2004-06-01"),"end-date":date("2010-03-09")}]}
+{"id":9039973,"id-copy":9039973,"alias":"Desmond","name":"DesmondRice","user-since":datetime("2008-04-17T12:00:38"),"user-since-copy":datetime("2008-04-17T12:00:38"),"friend-ids":{{16128090,28937536,30905098,25666304,23272582,29438991,42040849,42396891,9345677,9260055,17415621,31581557,1249365,20734436,2341357,36307325,20347771,23723655}},"employment":[{"organization-name":"Zimcone","start-date":date("2002-10-24"),"end-date":date("2008-02-24")}]}
+{"id":9503443,"id-copy":9503443,"alias":"Ebenezer","name":"EbenezerFulton","user-since":datetime("2012-07-03T20:14:05"),"user-since-copy":datetime("2012-07-03T20:14:05"),"friend-ids":{{11155403,7932344,24822329,19823943,37496284}},"employment":[{"organization-name":"Doublezone","start-date":date("2011-08-22")}]}
+{"id":9345424,"id-copy":9345424,"alias":"Jasmin","name":"JasminGaskins","user-since":datetime("2012-06-15T19:40:07"),"user-since-copy":datetime("2012-06-15T19:40:07"),"friend-ids":{{20837477,42339634,41136248,24571549,41060055,18621328,2057295,41313707}},"employment":[{"organization-name":"ganjalax","start-date":date("2012-05-27"),"end-date":date("2012-07-28")}]}
+{"id":10318882,"id-copy":10318882,"alias":"Skyler","name":"SkylerConrad","user-since":datetime("2007-03-04T08:56:54"),"user-since-copy":datetime("2007-03-04T08:56:54"),"friend-ids":{{4254240,3778434,23914534,16376376,39143316,37229152,32778982,30182686,13077652,20439638,34086734,12101909,47011547,28666460,31034524,47508299,17267782,1260337,43500601,914291,1786773}},"employment":[{"organization-name":"Villa-tech","start-date":date("2009-01-15")}]}
+{"id":10173691,"id-copy":10173691,"alias":"Elissa","name":"ElissaWilliams","user-since":datetime("2011-09-26T16:07:17"),"user-since-copy":datetime("2011-09-26T16:07:17"),"friend-ids":{{2526422}},"employment":[{"organization-name":"Whitemedia","start-date":date("2001-07-22")}]}
+{"id":11010904,"id-copy":11010904,"alias":"Chang","name":"ChangSteele","user-since":datetime("2009-02-24T01:43:56"),"user-since-copy":datetime("2009-02-24T01:43:56"),"friend-ids":{{19212881,4019921,24976558,47613555,26049623,17656988,24011085,31763054,21741933,31356824,9651386,35034682,5665574,31306405,38922156,9837341,31865250,12415354}},"employment":[{"organization-name":"freshdox","start-date":date("2005-09-20"),"end-date":date("2005-05-28")}]}
+{"id":10579345,"id-copy":10579345,"alias":"Rexana","name":"RexanaSchaeffer","user-since":datetime("2006-01-20T15:37:57"),"user-since-copy":datetime("2006-01-20T15:37:57"),"friend-ids":{{20070497,44547094,38571608,30731404,7825730,8433351,25090042,38943273,3599029,28517891,17427828,6853394,32856065,46627870,43885788}},"employment":[{"organization-name":"Ganjatax","start-date":date("2012-01-22")}]}
+{"id":9269422,"id-copy":9269422,"alias":"Roddy","name":"RoddyFriedline","user-since":datetime("2007-03-26T23:41:29"),"user-since-copy":datetime("2007-03-26T23:41:29"),"friend-ids":{{31923430,19739952,30983882,10630795}},"employment":[{"organization-name":"Zuncan","start-date":date("2001-06-03")}]}
+{"id":9083791,"id-copy":9083791,"alias":"Lashay","name":"LashayLeonard","user-since":datetime("2008-07-03T04:52:06"),"user-since-copy":datetime("2008-07-03T04:52:06"),"friend-ids":{{16762687,32021842,851915,36102981,3553783,30756474,12043049,16852621,35699568,4425852,35227725,25748188,9140215,24886626,1945167,12733697,20761965}},"employment":[{"organization-name":"Villa-dox","start-date":date("2003-05-19"),"end-date":date("2006-10-16")}]}
+{"id":11417764,"id-copy":11417764,"alias":"Maren","name":"MarenDickson","user-since":datetime("2006-07-20T06:36:52"),"user-since-copy":datetime("2006-07-20T06:36:52"),"friend-ids":{{14573904,11946003,35291176,25103717,30010131,886854,46625000,28533752,46506784,15300620,40647607,10249516,27751123,3883546,41772148,26655932,39026036,4416966,15070564,7052224,10264392,13650303,30752174}},"employment":[{"organization-name":"Canline","start-date":date("2012-08-26"),"end-date":date("2012-08-29")}]}
+{"id":11348356,"id-copy":11348356,"alias":"Chery","name":"CherySandford","user-since":datetime("2011-04-23T21:22:21"),"user-since-copy":datetime("2011-04-23T21:22:21"),"friend-ids":{{14076544,42221517}},"employment":[{"organization-name":"Freshfix","start-date":date("2000-07-25")}]}
+{"id":11518480,"id-copy":11518480,"alias":"Amada","name":"AmadaTanner","user-since":datetime("2006-05-06T12:27:31"),"user-since-copy":datetime("2006-05-06T12:27:31"),"friend-ids":{{}},"employment":[{"organization-name":"Dandamace","start-date":date("2002-04-02")}]}
+{"id":9677293,"id-copy":9677293,"alias":"Owen","name":"OwenHoenshell","user-since":datetime("2005-06-28T02:54:49"),"user-since-copy":datetime("2005-06-28T02:54:49"),"friend-ids":{{1016713,4999321,27107303,15587298}},"employment":[{"organization-name":"Fax-fax","start-date":date("2010-01-11")}]}
+{"id":11032477,"id-copy":11032477,"alias":"Wilmer","name":"WilmerWortman","user-since":datetime("2007-06-03T19:27:24"),"user-since-copy":datetime("2007-06-03T19:27:24"),"friend-ids":{{18685187,2599612,27305395,20825021,20327586,21301262,29222955,20377452,11211553,37446807,20533832,10098143,43828837,37254072,46029810,16401947,7537056,41738273,4665729,27400110,146251,14185116}},"employment":[{"organization-name":"Transhigh","start-date":date("2006-03-17"),"end-date":date("2011-08-03")}]}
+{"id":11362531,"id-copy":11362531,"alias":"Garey","name":"GareyChapman","user-since":datetime("2005-10-13T04:24:29"),"user-since-copy":datetime("2005-10-13T04:24:29"),"friend-ids":{{20693565,18896854,17118168,12285534,21434048,15453439,42734432,3627967,30464042,11556192,22808282,464074,28100870,29887664,19046987,34996619,39964690,22574200,29497238}},"employment":[{"organization-name":"Xx-technology","start-date":date("2001-03-05")}]}
+{"id":9819796,"id-copy":9819796,"alias":"Emerson","name":"EmersonWardle","user-since":datetime("2006-08-20T20:22:11"),"user-since-copy":datetime("2006-08-20T20:22:11"),"friend-ids":{{5697147,42936553,12624322,45309083,10785774,4176618}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2003-05-16")}]}
+{"id":11625859,"id-copy":11625859,"alias":"Zacharias","name":"ZachariasSanner","user-since":datetime("2007-06-12T21:21:21"),"user-since-copy":datetime("2007-06-12T21:21:21"),"friend-ids":{{13379571,45822651,39352555,11549959,24329960,2142134,15486962,43011509,46074449,9322703}},"employment":[{"organization-name":"Transhigh","start-date":date("2007-05-15")}]}
+{"id":9968869,"id-copy":9968869,"alias":"Shemika","name":"ShemikaNickolson","user-since":datetime("2005-02-20T10:34:04"),"user-since-copy":datetime("2005-02-20T10:34:04"),"friend-ids":{{30287118,877645,9968776,31800907}},"employment":[{"organization-name":"Tranzap","start-date":date("2009-07-23")}]}
+{"id":11223157,"id-copy":11223157,"alias":"Lavina","name":"LavinaPeters","user-since":datetime("2007-11-08T11:13:48"),"user-since-copy":datetime("2007-11-08T11:13:48"),"friend-ids":{{45286302}},"employment":[{"organization-name":"Ransaofan","start-date":date("2012-01-13")}]}
+{"id":9799591,"id-copy":9799591,"alias":"Royston","name":"RoystonChurchill","user-since":datetime("2011-01-21T13:57:31"),"user-since-copy":datetime("2011-01-21T13:57:31"),"friend-ids":{{22757950,4629721,19522595,27737642,39393176,9321441,13496995,43301849,3869585,34993450,24876688}},"employment":[{"organization-name":"Tripplelane","start-date":date("2000-07-15")}]}
+{"id":9950824,"id-copy":9950824,"alias":"Maryann","name":"MaryannCressman","user-since":datetime("2011-02-25T17:51:21"),"user-since-copy":datetime("2011-02-25T17:51:21"),"friend-ids":{{30203965,23348792,19093409,21079475}},"employment":[{"organization-name":"Rungozoom","start-date":date("2006-10-11"),"end-date":date("2006-10-09")}]}
+{"id":10205539,"id-copy":10205539,"alias":"Raeburn","name":"RaeburnWire","user-since":datetime("2007-04-28T23:05:24"),"user-since-copy":datetime("2007-04-28T23:05:24"),"friend-ids":{{13609724,40251506}},"employment":[{"organization-name":"Latsonity","start-date":date("2000-08-09")}]}
+{"id":9568750,"id-copy":9568750,"alias":"Daley","name":"DaleyHarshman","user-since":datetime("2012-01-17T10:38:07"),"user-since-copy":datetime("2012-01-17T10:38:07"),"friend-ids":{{18932212,37118057,37586464,12686041,21083532,27598912}},"employment":[{"organization-name":"Medflex","start-date":date("2007-11-07")}]}
+{"id":11763463,"id-copy":11763463,"alias":"Haven","name":"HavenRaub","user-since":datetime("2012-03-01T12:41:53"),"user-since-copy":datetime("2012-03-01T12:41:53"),"friend-ids":{{19981286}},"employment":[{"organization-name":"Rungozoom","start-date":date("2001-04-26")}]}
+{"id":10206877,"id-copy":10206877,"alias":"Tammie","name":"TammieBerry","user-since":datetime("2009-10-14T12:57:11"),"user-since-copy":datetime("2009-10-14T12:57:11"),"friend-ids":{{23748102,37944735,42193629,11409119,41246083,35024235}},"employment":[{"organization-name":"Solfix","start-date":date("2008-05-21")}]}
+{"id":10808932,"id-copy":10808932,"alias":"Sharita","name":"SharitaGregory","user-since":datetime("2006-09-17T04:48:23"),"user-since-copy":datetime("2006-09-17T04:48:23"),"friend-ids":{{41622567,16559791,6346693,18540237,14753253,23252825,17163196,46962665,26442426,14344279,17332246,36154890,22814241,22709064,32887290,42853122,23782934,27425228,22941847}},"employment":[{"organization-name":"Ontohothex","start-date":date("2008-06-08"),"end-date":date("2011-01-28")}]}
+{"id":9099376,"id-copy":9099376,"alias":"Tena","name":"TenaKline","user-since":datetime("2011-10-20T14:46:29"),"user-since-copy":datetime("2011-10-20T14:46:29"),"friend-ids":{{28615752,16589994,24896126,32768352,40921310,22643822,39206554,45652466,17237997,44705249,30599864,17750741,14758376,4842744}},"employment":[{"organization-name":"Inchex","start-date":date("2000-03-18")}]}
+{"id":9563056,"id-copy":9563056,"alias":"Iantha","name":"IanthaHoward","user-since":datetime("2009-03-09T10:16:12"),"user-since-copy":datetime("2009-03-09T10:16:12"),"friend-ids":{{31445918,39207727,45365035,7861010,28533268,29009652,40156013,40416479,42741676,30221879,30189614,46450645,30914117,33681301,19457868,23309378,15126664,32913981,5396205}},"employment":[{"organization-name":"Unijobam","start-date":date("2000-03-18"),"end-date":date("2009-01-05")}]}
+{"id":11886709,"id-copy":11886709,"alias":"Leigh","name":"LeighBatten","user-since":datetime("2005-06-18T21:25:13"),"user-since-copy":datetime("2005-06-18T21:25:13"),"friend-ids":{{161610,3498914,24173074,33102324,42213688,44551300,36373040,30704767,24224319,5784194,13092764,38315503,13246046,2836280,672136,37021775}},"employment":[{"organization-name":"Greencare","start-date":date("2001-05-26"),"end-date":date("2001-05-11")}]}
+{"id":10765090,"id-copy":10765090,"alias":"Louiza","name":"LouizaMcelroy","user-since":datetime("2012-08-14T02:46:00"),"user-since-copy":datetime("2012-08-14T02:46:00"),"friend-ids":{{14365973,9091111,44279279,45125689,29955385,23874606,18142514,24878700,13928633,47391704,29729670,35422059,987030,3200788,7640346,32947024,32550247,25746061,34112521,41193622,2620213,30090329,5531715}},"employment":[{"organization-name":"Keytech","start-date":date("2002-06-16"),"end-date":date("2003-05-13")}]}
+{"id":10642153,"id-copy":10642153,"alias":"Wally","name":"WallyRiggle","user-since":datetime("2011-10-10T21:43:33"),"user-since-copy":datetime("2011-10-10T21:43:33"),"friend-ids":{{32910135,45556839,6526394,13177451,10588491,40270322,17438379,21204776,46036116,44249789,7375979,43487252,24858016,3947997}},"employment":[{"organization-name":"Ontotanin","start-date":date("2001-10-10")}]}
+{"id":11474374,"id-copy":11474374,"alias":"Waldo","name":"WaldoKnapp","user-since":datetime("2008-08-17T21:17:28"),"user-since-copy":datetime("2008-08-17T21:17:28"),"friend-ids":{{33358772,16499546,8631001,6045567,45554236,36229482,354579,11884970,23657774,32568373}},"employment":[{"organization-name":"Ontotanin","start-date":date("2004-11-18")}]}
+{"id":10492168,"id-copy":10492168,"alias":"Savannah","name":"SavannahRobinson","user-since":datetime("2008-05-02T04:19:01"),"user-since-copy":datetime("2008-05-02T04:19:01"),"friend-ids":{{40126719,38171650,1474355,6983398,7918678,45578368,3210188,29374863,37758187,2415003,13746140,44168763,45798029,17203664,46309082,21338452,17217009,24916114}},"employment":[{"organization-name":"Quoline","start-date":date("2009-07-20"),"end-date":date("2009-03-01")}]}
+{"id":10847359,"id-copy":10847359,"alias":"Leone","name":"LeoneWood","user-since":datetime("2005-07-28T14:24:43"),"user-since-copy":datetime("2005-07-28T14:24:43"),"friend-ids":{{7650486,39843416,43272193,47152762,45218041,45422234,46812876,18098636,47174431,19091549,1405281,46699360,37961345,43323551,46824225,30700451,10188790,16642374,26570751}},"employment":[{"organization-name":"Villa-dox","start-date":date("2005-01-22")}]}
+{"id":10047373,"id-copy":10047373,"alias":"Rexana","name":"RexanaDennis","user-since":datetime("2010-01-05T15:43:34"),"user-since-copy":datetime("2010-01-05T15:43:34"),"friend-ids":{{1594,40130182}},"employment":[{"organization-name":"freshdox","start-date":date("2004-07-04"),"end-date":date("2007-12-28")}]}
+{"id":10734148,"id-copy":10734148,"alias":"Allannah","name":"AllannahHoffhants","user-since":datetime("2005-11-18T00:54:25"),"user-since-copy":datetime("2005-11-18T00:54:25"),"friend-ids":{{26897353,13343289,1991130,39024681,21839148,38693973,19132058,17589948,13367008,30389658,21757614,45618415,23559236,35669455,22088928,2531202,120534,867017,8590987,25956219,21819960,41918122,31042839,15019901}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2004-10-25")}]}
+{"id":10948003,"id-copy":10948003,"alias":"August","name":"AugustHatch","user-since":datetime("2006-04-11T03:32:56"),"user-since-copy":datetime("2006-04-11T03:32:56"),"friend-ids":{{}},"employment":[{"organization-name":"Lexitechno","start-date":date("2008-12-16"),"end-date":date("2009-01-21")}]}
+{"id":10367416,"id-copy":10367416,"alias":"Damion","name":"DamionDean","user-since":datetime("2008-01-06T05:55:09"),"user-since-copy":datetime("2008-01-06T05:55:09"),"friend-ids":{{45804001,13077962,28346489,25877214,10164033,42903493,66753,27961850,41137249,20490506}},"employment":[{"organization-name":"geomedia","start-date":date("2007-12-18")}]}
+{"id":10365688,"id-copy":10365688,"alias":"Innocent","name":"InnocentBlatenberger","user-since":datetime("2008-11-09T13:57:34"),"user-since-copy":datetime("2008-11-09T13:57:34"),"friend-ids":{{27902413,27226238,35017422,28154221}},"employment":[{"organization-name":"Technohow","start-date":date("2011-09-13"),"end-date":date("2011-02-05")}]}
+{"id":11616502,"id-copy":11616502,"alias":"Bernetta","name":"BernettaMackendoerfer","user-since":datetime("2005-04-22T03:41:17"),"user-since-copy":datetime("2005-04-22T03:41:17"),"friend-ids":{{18804036,29570084,43932411,41492349,46505981,32524166,5307968}},"employment":[{"organization-name":"Salthex","start-date":date("2004-08-14"),"end-date":date("2009-08-03")}]}
+{"id":9552016,"id-copy":9552016,"alias":"Shantelle","name":"ShantelleDealtry","user-since":datetime("2006-05-03T06:49:13"),"user-since-copy":datetime("2006-05-03T06:49:13"),"friend-ids":{{35758396,16562240,23596680,16342769,19892813,46485447,25711418,23765073,11303996,36451291,17586370,38010455,29457199,25847013,12604123,46533018,26999208,24740610,35225441,33613663}},"employment":[{"organization-name":"Xx-technology","start-date":date("2003-08-07"),"end-date":date("2003-07-17")}]}
+{"id":10912441,"id-copy":10912441,"alias":"Janae","name":"JanaeErschoff","user-since":datetime("2009-04-17T09:26:36"),"user-since-copy":datetime("2009-04-17T09:26:36"),"friend-ids":{{11445243,13239218,2302326,37976140,45374131,14136536,2051767,7824391,42808044,41836900,35275542,33493951,8497237,42991362,24049395,32159562,23378256,4723574,47010157}},"employment":[{"organization-name":"zoomplus","start-date":date("2012-04-20"),"end-date":date("2012-04-04")}]}
+{"id":9209866,"id-copy":9209866,"alias":"Timothy","name":"TimothyBuck","user-since":datetime("2009-11-07T14:19:12"),"user-since-copy":datetime("2009-11-07T14:19:12"),"friend-ids":{{43082021,25019103,26061770,7134151,17663441,35230064,731481,6719229,23303796,40777269}},"employment":[{"organization-name":"U-electrics","start-date":date("2000-04-03"),"end-date":date("2000-04-20")}]}
+{"id":11709478,"id-copy":11709478,"alias":"Jonty","name":"JontyCurry","user-since":datetime("2006-09-08T22:15:05"),"user-since-copy":datetime("2006-09-08T22:15:05"),"friend-ids":{{1684909,3914449,16704128,11890093,44073634,24897496}},"employment":[{"organization-name":"itlab","start-date":date("2006-03-01")}]}
+{"id":11886856,"id-copy":11886856,"alias":"Eldred","name":"EldredArmstrong","user-since":datetime("2012-02-20T10:08:40"),"user-since-copy":datetime("2012-02-20T10:08:40"),"friend-ids":{{5146204,10549788,40744824,38277859}},"employment":[{"organization-name":"Canline","start-date":date("2006-09-18")}]}
+{"id":10799674,"id-copy":10799674,"alias":"Dolores","name":"DoloresPolson","user-since":datetime("2006-03-24T00:54:47"),"user-since-copy":datetime("2006-03-24T00:54:47"),"friend-ids":{{40482317,21393644,151122,13958566,6524741,1269094,34703787,38215473,20258639,144407,23903205,46922014,26741209,34932062,1043581,14090176,45243069,19226320,33271281,20215000,46383495,42405679,42360649}},"employment":[{"organization-name":"Voltlane","start-date":date("2009-07-18")}]}
+{"id":11230663,"id-copy":11230663,"alias":"Caryl","name":"CarylSmail","user-since":datetime("2006-03-17T16:52:51"),"user-since-copy":datetime("2006-03-17T16:52:51"),"friend-ids":{{32153460,21186863,24199212,25220508,26590053,42433121,35372685}},"employment":[{"organization-name":"Latsonity","start-date":date("2007-12-05")}]}
+{"id":9805759,"id-copy":9805759,"alias":"Emmie","name":"EmmieJohns","user-since":datetime("2008-11-01T15:15:13"),"user-since-copy":datetime("2008-11-01T15:15:13"),"friend-ids":{{47090234,24484835,11048702}},"employment":[{"organization-name":"Villa-dox","start-date":date("2008-02-26")}]}
+{"id":10251805,"id-copy":10251805,"alias":"Jericho","name":"JerichoBaird","user-since":datetime("2005-07-02T12:57:18"),"user-since-copy":datetime("2005-07-02T12:57:18"),"friend-ids":{{5748549,47013396,15858292,458526,28324553,22401875,21726858,38878600,29844738,14547049,11432495,9227475}},"employment":[{"organization-name":"goldendexon","start-date":date("2005-06-09"),"end-date":date("2011-11-01")}]}
+{"id":9744016,"id-copy":9744016,"alias":"Kasha","name":"KashaMueller","user-since":datetime("2011-03-16T17:17:31"),"user-since-copy":datetime("2011-03-16T17:17:31"),"friend-ids":{{15857660,46791109,10310040,42863950,19533508,32561502,4367358,31952243,7130063,19536081,19870534,3642001,910385,28668446,33204842,13210089,2805429}},"employment":[{"organization-name":"Labzatron","start-date":date("2000-11-01")}]}
+{"id":10197700,"id-copy":10197700,"alias":"Frederica","name":"FredericaCherry","user-since":datetime("2006-04-10T01:23:53"),"user-since-copy":datetime("2006-04-10T01:23:53"),"friend-ids":{{}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-01-01"),"end-date":date("2009-07-14")}]}
+{"id":11713315,"id-copy":11713315,"alias":"Chung","name":"ChungStroble","user-since":datetime("2005-10-20T22:59:27"),"user-since-copy":datetime("2005-10-20T22:59:27"),"friend-ids":{{13105744,9160760,37104436,33688116,31455484,44428287}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2002-12-03"),"end-date":date("2010-10-06")}]}
+{"id":11830663,"id-copy":11830663,"alias":"Bettie","name":"BettieKing","user-since":datetime("2009-11-06T15:04:55"),"user-since-copy":datetime("2009-11-06T15:04:55"),"friend-ids":{{46068058,35215092,34850678,9126970,16472040,20000261,17610567,37016763,19830405,38071058,43961371,13092410,24867008,12366628,15539063,15611017,1343975,43254018,30838755,30488641,38027133,5701592}},"employment":[{"organization-name":"subtam","start-date":date("2003-04-10")}]}
+{"id":11945014,"id-copy":11945014,"alias":"Lavern","name":"LavernRahl","user-since":datetime("2005-08-13T08:07:58"),"user-since-copy":datetime("2005-08-13T08:07:58"),"friend-ids":{{15127940,37543274,13877909,8961585,13712343,38178056,21469501,2994082,24368304,33508930,41765591,37858577,42295002}},"employment":[{"organization-name":"U-electrics","start-date":date("2001-07-20")}]}
+{"id":11779591,"id-copy":11779591,"alias":"Galina","name":"GalinaRoberts","user-since":datetime("2007-03-18T12:09:38"),"user-since-copy":datetime("2007-03-18T12:09:38"),"friend-ids":{{16134690,41543844}},"employment":[{"organization-name":"Xx-technology","start-date":date("2010-04-17")}]}
+{"id":11416066,"id-copy":11416066,"alias":"Janna","name":"JannaBowchiew","user-since":datetime("2010-12-06T10:53:56"),"user-since-copy":datetime("2010-12-06T10:53:56"),"friend-ids":{{43816151,22032304,27239988,23813127,34936097,8817657,39872787,27628236,38333824,40879066}},"employment":[{"organization-name":"U-electrics","start-date":date("2001-04-19"),"end-date":date("2008-01-09")}]}
+{"id":10789207,"id-copy":10789207,"alias":"Lucinda","name":"LucindaFillmore","user-since":datetime("2009-11-13T18:35:41"),"user-since-copy":datetime("2009-11-13T18:35:41"),"friend-ids":{{10917581,24902161,29393856,35293349,31477965,44139676,18083704,46487557}},"employment":[{"organization-name":"Ontohothex","start-date":date("2005-11-04")}]}
+{"id":11174689,"id-copy":11174689,"alias":"Thao","name":"ThaoBrandenburg","user-since":datetime("2012-04-21T05:25:58"),"user-since-copy":datetime("2012-04-21T05:25:58"),"friend-ids":{{37540210,3918403,33043564,33664166}},"employment":[{"organization-name":"Fix-touch","start-date":date("2001-08-22"),"end-date":date("2004-11-19")}]}
+{"id":9795463,"id-copy":9795463,"alias":"Brunilda","name":"BrunildaPheleps","user-since":datetime("2007-04-21T01:56:02"),"user-since-copy":datetime("2007-04-21T01:56:02"),"friend-ids":{{39507879,43296507,45019669,39481546,16657717,8707249,47148318,46560087,42473978,11974026,40145543,2127794,19537942,28159963,21439105,32578039,24112998,47853039,6406099,30697429}},"employment":[{"organization-name":"Latsonity","start-date":date("2001-07-13")}]}
+{"id":11290987,"id-copy":11290987,"alias":"Ilana","name":"IlanaTedrow","user-since":datetime("2009-03-03T00:10:34"),"user-since-copy":datetime("2009-03-03T00:10:34"),"friend-ids":{{20902982,27972021,22354642,32382609,18711912,17070293}},"employment":[{"organization-name":"physcane","start-date":date("2005-11-28"),"end-date":date("2009-09-17")}]}
+{"id":11290870,"id-copy":11290870,"alias":"Lanford","name":"LanfordOsteen","user-since":datetime("2009-03-04T15:04:12"),"user-since-copy":datetime("2009-03-04T15:04:12"),"friend-ids":{{4397941,36140649,12796618,18235191,8810154,10521988,6580979,29578654,46083953,30113784,25952539}},"employment":[{"organization-name":"highfax","start-date":date("2009-08-06")}]}
+{"id":10739446,"id-copy":10739446,"alias":"Urban","name":"UrbanHair","user-since":datetime("2010-12-28T02:29:19"),"user-since-copy":datetime("2010-12-28T02:29:19"),"friend-ids":{{31947556,39058269,43315882,40575729,4079275,40689246,22639555,1422452,28051313,41854009,30810426,37406811,20834349,46933622,28218698,17239481,33458180}},"employment":[{"organization-name":"freshdox","start-date":date("2006-05-21")}]}
+{"id":9490342,"id-copy":9490342,"alias":"Gisela","name":"GiselaTomlinson","user-since":datetime("2011-10-21T20:36:09"),"user-since-copy":datetime("2011-10-21T20:36:09"),"friend-ids":{{27609144,42495049,21250269,22561106,29149509,16776721,16980559,19600765}},"employment":[{"organization-name":"Viatechi","start-date":date("2003-02-23")}]}
+{"id":11515915,"id-copy":11515915,"alias":"Hunter","name":"HunterBash","user-since":datetime("2011-03-05T16:16:17"),"user-since-copy":datetime("2011-03-05T16:16:17"),"friend-ids":{{14847122,46314922,14414318,46374290,45050391,22617753}},"employment":[{"organization-name":"Solfix","start-date":date("2004-01-20")}]}
+{"id":9996817,"id-copy":9996817,"alias":"Vere","name":"VereWilkerson","user-since":datetime("2012-02-05T22:05:44"),"user-since-copy":datetime("2012-02-05T22:05:44"),"friend-ids":{{30010110,31604568,5741065,29161468,22429704,16954129,26525860,1490181,11444321,24455724,10411850,39851031,16059860,32050795,13116007,12071588}},"employment":[{"organization-name":"Ganjatax","start-date":date("2004-11-04")}]}
+{"id":10567702,"id-copy":10567702,"alias":"Zelda","name":"ZeldaRitter","user-since":datetime("2010-09-27T12:52:54"),"user-since-copy":datetime("2010-09-27T12:52:54"),"friend-ids":{{28336161,20248788,24723848,8856879,16831898,7643547,42868543,23023606,7531861,36186817,29113040,995506,32607538,18755505,44683178,24627205,39736850,43535271,385416,40525568}},"employment":[{"organization-name":"Indiex","start-date":date("2011-11-27"),"end-date":date("2011-08-16")}]}
+{"id":11525302,"id-copy":11525302,"alias":"Marissa","name":"MarissaEndsley","user-since":datetime("2006-09-26T08:55:36"),"user-since-copy":datetime("2006-09-26T08:55:36"),"friend-ids":{{35476434,12502442,19198691,35401830,14414490,11372357,28886265,3490052,13587860,8127851,20732439,44816539,6616740,12785784,16907259,10942007,26207,21026660,39284170,25761798,20688453,45805952,15912564}},"employment":[{"organization-name":"Latsonity","start-date":date("2007-10-07"),"end-date":date("2010-09-09")}]}
+{"id":11147392,"id-copy":11147392,"alias":"Sarina","name":"SarinaFlickinger","user-since":datetime("2011-09-26T12:41:56"),"user-since-copy":datetime("2011-09-26T12:41:56"),"friend-ids":{{17776087,9254087,14735666,31097664,36421253,12595115,40366588,9491701,29725314,38852857,46206259,39281843,36268114,29939350,804107,36307361,30999436,47369074,3820973,46362092,36413930,8807546,30260636,15069463}},"employment":[{"organization-name":"Ganjastrip","start-date":date("2005-03-15")}]}
+{"id":10122346,"id-copy":10122346,"alias":"Salal","name":"SalalPearson","user-since":datetime("2011-11-14T10:42:11"),"user-since-copy":datetime("2011-11-14T10:42:11"),"friend-ids":{{44003884,37124809,7600567,5158911,31009406,10708460}},"employment":[{"organization-name":"Newcom","start-date":date("2001-02-17"),"end-date":date("2010-06-23")}]}
+{"id":9461770,"id-copy":9461770,"alias":"Georgina","name":"GeorginaPearson","user-since":datetime("2005-02-04T09:47:21"),"user-since-copy":datetime("2005-02-04T09:47:21"),"friend-ids":{{26615251,5874803,5189465,29564778,1778424,38706542,38915757,16819394,3318129,2166806,30570432,15192853,4857015,41673300,23510020}},"employment":[{"organization-name":"Dancode","start-date":date("2003-06-06")}]}
+{"id":9041689,"id-copy":9041689,"alias":"Freeman","name":"FreemanDriggers","user-since":datetime("2011-05-23T03:51:13"),"user-since-copy":datetime("2011-05-23T03:51:13"),"friend-ids":{{29448942,29196543,22725448,15145190,11938396,44028947,18379392,21813464,7448397,43717728,10728731,24177517,29069798,37056934,27601399,26867839,16593922,22247111}},"employment":[{"organization-name":"silfind","start-date":date("2007-01-14")}]}
+{"id":10479073,"id-copy":10479073,"alias":"Rhianna","name":"RhiannaWerry","user-since":datetime("2009-09-17T19:42:47"),"user-since-copy":datetime("2009-09-17T19:42:47"),"friend-ids":{{30293616,42971604,8411318,37648744,27412687,17821200,45008072}},"employment":[{"organization-name":"Ontohothex","start-date":date("2004-11-02"),"end-date":date("2011-06-24")}]}
+{"id":11309383,"id-copy":11309383,"alias":"Lyn","name":"LynKnapp","user-since":datetime("2010-07-21T15:29:58"),"user-since-copy":datetime("2010-07-21T15:29:58"),"friend-ids":{{27610153}},"employment":[{"organization-name":"Tanzumbam","start-date":date("2012-08-28"),"end-date":date("2012-08-29")}]}
+{"id":11004067,"id-copy":11004067,"alias":"Vickie","name":"VickieRosenstiehl","user-since":datetime("2012-04-15T02:37:43"),"user-since-copy":datetime("2012-04-15T02:37:43"),"friend-ids":{{}},"employment":[{"organization-name":"Inchex","start-date":date("2012-08-01"),"end-date":date("2012-08-06")}]}
+{"id":9369847,"id-copy":9369847,"alias":"Jeffrey","name":"JeffreyArchibald","user-since":datetime("2011-07-11T23:43:52"),"user-since-copy":datetime("2011-07-11T23:43:52"),"friend-ids":{{44928062,45653705}},"employment":[{"organization-name":"Unijobam","start-date":date("2010-03-25")}]}
+{"id":11670739,"id-copy":11670739,"alias":"Rudyard","name":"RudyardErrett","user-since":datetime("2005-03-08T18:26:12"),"user-since-copy":datetime("2005-03-08T18:26:12"),"friend-ids":{{13253132,38903405,45479471,11551894,44803858,34016119,2477206,27909363,2584557,29078732,13687500,1038800,14467502,3369722,11731177,15702876,37034289,21943459}},"employment":[{"organization-name":"Indiex","start-date":date("2011-01-20")}]}
+{"id":10222144,"id-copy":10222144,"alias":"Alvina","name":"AlvinaTanner","user-since":datetime("2007-10-15T04:24:14"),"user-since-copy":datetime("2007-10-15T04:24:14"),"friend-ids":{{44207447,29837430,407059,4562324,970458,31348025,16439061,13011150,23510630,21529259,8279487,28052530,36551405,17492050,17983056,11834104,242520,9279232,4179609,28407763,23038009,36977762,8779957,15040402}},"employment":[{"organization-name":"Ronholdings","start-date":date("2006-05-27")}]}
+{"id":9917008,"id-copy":9917008,"alias":"Clancy","name":"ClancyHector","user-since":datetime("2007-09-25T20:55:57"),"user-since-copy":datetime("2007-09-25T20:55:57"),"friend-ids":{{37754545,37579706,39121342,28434988,3927416,3794736,17107964,20761621,20497172,28562441,4310488,35121288,2380560,32434056}},"employment":[{"organization-name":"Vaiatech","start-date":date("2011-06-24")}]}
+{"id":11327029,"id-copy":11327029,"alias":"Mallory","name":"MalloryHughes","user-since":datetime("2007-08-06T22:11:46"),"user-since-copy":datetime("2007-08-06T22:11:46"),"friend-ids":{{38924183,22042572,21014848,46309217,1120998,19755064,4413438,38855205,17626985,5727472,1293238}},"employment":[{"organization-name":"Solfix","start-date":date("2006-02-28"),"end-date":date("2006-08-24")}]}
+{"id":9275620,"id-copy":9275620,"alias":"Jackie","name":"JackieRumbaugh","user-since":datetime("2011-10-11T07:30:25"),"user-since-copy":datetime("2011-10-11T07:30:25"),"friend-ids":{{}},"employment":[{"organization-name":"Technohow","start-date":date("2004-01-14")}]}
+{"id":10915261,"id-copy":10915261,"alias":"Lyle","name":"LyleMuller","user-since":datetime("2010-10-16T16:36:46"),"user-since-copy":datetime("2010-10-16T16:36:46"),"friend-ids":{{28409003,7495999,10776059,23825626,44321306,15679301,36736470,24070644,14041140,4784196,19462533,47300197,33544003}},"employment":[{"organization-name":"Hexviafind","start-date":date("2006-09-25")}]}
+{"id":9598486,"id-copy":9598486,"alias":"Grover","name":"GroverNewbern","user-since":datetime("2012-01-06T20:50:38"),"user-since-copy":datetime("2012-01-06T20:50:38"),"friend-ids":{{8389292,25521744,23387036,38008541,43673600,23656679,1401712,39164079,1810015,20625744,15651316,23441546,24572830,19077921}},"employment":[{"organization-name":"Tripplelane","start-date":date("2009-07-28"),"end-date":date("2010-06-09")}]}
+{"id":10548142,"id-copy":10548142,"alias":"Dannie","name":"DannieTillson","user-since":datetime("2007-03-07T04:57:23"),"user-since-copy":datetime("2007-03-07T04:57:23"),"friend-ids":{{37443492,21615683,5655492,24162015,46418787,46328489,26669127,38324141}},"employment":[{"organization-name":"Voltlane","start-date":date("2012-01-03")}]}
+{"id":9665848,"id-copy":9665848,"alias":"Shannah","name":"ShannahDale","user-since":datetime("2006-08-09T02:09:51"),"user-since-copy":datetime("2006-08-09T02:09:51"),"friend-ids":{{19512022,25217933,21742776,35558948,5893317,2441637,6907563,36626257,3366834,25069218,5753530,45604388,33908296,1048890,5720452,7923351,43424884,43184720,29744229,10349400,15273614,15283237,41997307}},"employment":[{"organization-name":"Fix-touch","start-date":date("2010-12-28"),"end-date":date("2010-09-17")}]}
+{"id":9885289,"id-copy":9885289,"alias":"Kayla","name":"KaylaDugger","user-since":datetime("2007-10-20T12:55:38"),"user-since-copy":datetime("2007-10-20T12:55:38"),"friend-ids":{{1821427,46609485,4532131}},"employment":[{"organization-name":"Ontotanin","start-date":date("2009-02-15"),"end-date":date("2009-11-17")}]}
+{"id":9674677,"id-copy":9674677,"alias":"Skye","name":"SkyeTomlinson","user-since":datetime("2006-02-02T19:15:10"),"user-since-copy":datetime("2006-02-02T19:15:10"),"friend-ids":{{24282798,5600117,33292938,19518197,11735189,22867735,8029689,11269147,7443311,45905216,12859442,26944030}},"employment":[{"organization-name":"whitestreet","start-date":date("2011-05-07"),"end-date":date("2011-04-19")}]}
+{"id":10384705,"id-copy":10384705,"alias":"Santos","name":"SantosJames","user-since":datetime("2011-05-07T11:54:13"),"user-since-copy":datetime("2011-05-07T11:54:13"),"friend-ids":{{43937179,34015979,7417213,14660995,19725400,3931428,7318379,48016396,44068471,4577462,38302695,16520658,40487183,31181305,11750148,42688348,42071075,10641987,28860865,27686448,40844612,10817134}},"employment":[{"organization-name":"Ronholdings","start-date":date("2007-12-18")}]}
+{"id":9446506,"id-copy":9446506,"alias":"Deshawn","name":"DeshawnBashline","user-since":datetime("2009-03-11T18:09:19"),"user-since-copy":datetime("2009-03-11T18:09:19"),"friend-ids":{{22236205,44669386,5098679,17631352,40353783,17155709}},"employment":[{"organization-name":"Freshfix","start-date":date("2002-11-12"),"end-date":date("2003-04-22")}]}
+{"id":10888777,"id-copy":10888777,"alias":"Bevis","name":"BevisStall","user-since":datetime("2007-04-05T02:35:27"),"user-since-copy":datetime("2007-04-05T02:35:27"),"friend-ids":{{1924847,33036971,5163765,37816368,15975671,11388174,38485519,43186487,30402693,34350975,24348537,34349089,22680019,30625064,23751465,9072515,15915109}},"employment":[{"organization-name":"Ontotanin","start-date":date("2005-03-11")}]}
+{"id":11012734,"id-copy":11012734,"alias":"Jordan","name":"JordanSadley","user-since":datetime("2011-02-26T18:40:19"),"user-since-copy":datetime("2011-02-26T18:40:19"),"friend-ids":{{37319587,37212468,3023956,43125609}},"employment":[{"organization-name":"U-ron","start-date":date("2007-07-03"),"end-date":date("2011-01-25")}]}
+{"id":9082201,"id-copy":9082201,"alias":"Alberic","name":"AlbericCrawford","user-since":datetime("2005-02-11T07:41:05"),"user-since-copy":datetime("2005-02-11T07:41:05"),"friend-ids":{{26925567,6108069,30484049,4903722,4579631,21166966,3892344,6259030,32887933,7183018,46041497,23448710,47887528,3679587,7140571,47671072,4554470,23481403,16738975,4885244}},"employment":[{"organization-name":"Voltbam","start-date":date("2006-10-10")}]}
+{"id":9386794,"id-copy":9386794,"alias":"Issac","name":"IssacNickolson","user-since":datetime("2009-12-11T08:40:10"),"user-since-copy":datetime("2009-12-11T08:40:10"),"friend-ids":{{4077760,26197904,22088648}},"employment":[{"organization-name":"Fax-fax","start-date":date("2002-05-17")}]}
+{"id":10856059,"id-copy":10856059,"alias":"Leland","name":"LelandMcdonald","user-since":datetime("2006-09-26T03:35:07"),"user-since-copy":datetime("2006-09-26T03:35:07"),"friend-ids":{{29735881,7080599,14172811,24274797,5773081,2653240,18151967,34988676,6599030,46463015,23254278,37618443,32396573}},"employment":[{"organization-name":"Inchex","start-date":date("2004-01-17")}]}
+{"id":11529952,"id-copy":11529952,"alias":"Charles","name":"CharlesHarrow","user-since":datetime("2008-11-24T19:27:12"),"user-since-copy":datetime("2008-11-24T19:27:12"),"friend-ids":{{}},"employment":[{"organization-name":"Keytech","start-date":date("2005-06-08"),"end-date":date("2011-10-27")}]}
+{"id":9854788,"id-copy":9854788,"alias":"Mathilda","name":"MathildaVanleer","user-since":datetime("2007-01-05T08:45:07"),"user-since-copy":datetime("2007-01-05T08:45:07"),"friend-ids":{{20510022,1353061,24801201,11438611,30281530,15596343,29404248,2024925,3425369,18530400}},"employment":[{"organization-name":"Zuncan","start-date":date("2002-07-22"),"end-date":date("2011-02-24")}]}
+{"id":10933138,"id-copy":10933138,"alias":"Gwendoline","name":"GwendolineCypret","user-since":datetime("2006-04-10T03:55:29"),"user-since-copy":datetime("2006-04-10T03:55:29"),"friend-ids":{{9996028,18756914,15079751,34129343,44558538,25387070,44250368,37560291,5178625,10379959,39639296,8784216,13429736,22802431,11154064,2453387,24748342,34032462,32570963,4861587,19421488,10848442}},"employment":[{"organization-name":"overtech","start-date":date("2008-12-24"),"end-date":date("2010-05-20")}]}
+{"id":11116465,"id-copy":11116465,"alias":"Read","name":"ReadOppenheimer","user-since":datetime("2012-08-23T03:38:20"),"user-since-copy":datetime("2012-08-23T03:38:20"),"friend-ids":{{18679034,12828526,13510152,28052139,20367021,30392195,41580515,2644015,29573423,22838698}},"employment":[{"organization-name":"tresline","start-date":date("2009-11-05")}]}
+{"id":10215280,"id-copy":10215280,"alias":"Barbara","name":"BarbaraEve","user-since":datetime("2012-03-09T01:36:52"),"user-since-copy":datetime("2012-03-09T01:36:52"),"friend-ids":{{32562793,33679771,10306498,37847497,30180151,3504698}},"employment":[{"organization-name":"Zuncan","start-date":date("2011-12-14")}]}
+{"id":11886532,"id-copy":11886532,"alias":"Tel","name":"TelGardner","user-since":datetime("2009-10-06T10:33:32"),"user-since-copy":datetime("2009-10-06T10:33:32"),"friend-ids":{{37243107,36561786,3939621,13531917,7768514,31689833,27145019,9462172,40579935,32184519,8668855,26137893,5582080,4847233,10244448,42634758,34911290,10834989,34800551,14109743}},"employment":[{"organization-name":"Coneflex","start-date":date("2010-07-24")}]}
+{"id":9740008,"id-copy":9740008,"alias":"Woodrow","name":"WoodrowBlois","user-since":datetime("2011-12-18T11:34:56"),"user-since-copy":datetime("2011-12-18T11:34:56"),"friend-ids":{{1753941,17603348,44569557,6816408,17403631,29707555,21215516,9837919,35887854,35236051,7897485,9880491,16145458,33128036,41471362,44171952,23542112,36155237,2596261,36702766}},"employment":[{"organization-name":"Statcode","start-date":date("2007-08-02")}]}
+{"id":9707074,"id-copy":9707074,"alias":"Melvyn","name":"MelvynSybilla","user-since":datetime("2012-06-07T16:06:49"),"user-since-copy":datetime("2012-06-07T16:06:49"),"friend-ids":{{4487400,488933,15650706,44692005,25068052,16975927}},"employment":[{"organization-name":"Villa-dox","start-date":date("2010-12-13")}]}
+{"id":9477994,"id-copy":9477994,"alias":"Cory","name":"CoryKeener","user-since":datetime("2012-02-27T22:03:31"),"user-since-copy":datetime("2012-02-27T22:03:31"),"friend-ids":{{22204843,35394804,22795967,16575437,31764908,27359073,50023,26383393,36534917,23478654,31022293,43803666,24764841,19469389,6401330,10543085,5159571}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2012-02-09"),"end-date":date("2012-02-19")}]}
+{"id":11919640,"id-copy":11919640,"alias":"Blanch","name":"BlanchHawkins","user-since":datetime("2007-09-24T10:11:40"),"user-since-copy":datetime("2007-09-24T10:11:40"),"friend-ids":{{28731986,7289796,42121816,33230171}},"employment":[{"organization-name":"Quoline","start-date":date("2007-09-17")}]}
+{"id":10271479,"id-copy":10271479,"alias":"Leah","name":"LeahKoepple","user-since":datetime("2007-10-26T15:57:39"),"user-since-copy":datetime("2007-10-26T15:57:39"),"friend-ids":{{317362,43304286,35630504,16014770,43567734,37946435,7728583,45620359,43235478,17133820,22926471,27438784,43521614,235789,43107565,21967424,39119573,1688079,5463246,10081045}},"employment":[{"organization-name":"Newphase","start-date":date("2012-06-14")}]}
+{"id":9882241,"id-copy":9882241,"alias":"Dillon","name":"DillonSimpson","user-since":datetime("2006-03-20T13:21:16"),"user-since-copy":datetime("2006-03-20T13:21:16"),"friend-ids":{{22747996,6266176,22832223,30880579,35481343,48005259,381757,27560756,6053858,42532723,33355330,40374460,39019469,35869327}},"employment":[{"organization-name":"Tanzumbam","start-date":date("2007-06-13"),"end-date":date("2011-08-15")}]}
+{"id":11840218,"id-copy":11840218,"alias":"Deandre","name":"DeandreMackendrick","user-since":datetime("2012-07-03T08:22:13"),"user-since-copy":datetime("2012-07-03T08:22:13"),"friend-ids":{{36310775,13455844,1133499,44183463,28002311,40758157,33299342,47526543,9613784,5698202,1492720,5663846}},"employment":[{"organization-name":"ganjalax","start-date":date("2006-03-12"),"end-date":date("2009-08-08")}]}
+{"id":10911274,"id-copy":10911274,"alias":"Bridgette","name":"BridgetteBenford","user-since":datetime("2007-02-15T06:18:45"),"user-since-copy":datetime("2007-02-15T06:18:45"),"friend-ids":{{10909520,14433605}},"employment":[{"organization-name":"Ganjastrip","start-date":date("2012-01-14")}]}
+{"id":10195063,"id-copy":10195063,"alias":"Rose","name":"RoseHatcher","user-since":datetime("2008-10-11T02:17:54"),"user-since-copy":datetime("2008-10-11T02:17:54"),"friend-ids":{{9820231,12294967,46911959,47936560,7881400,11585414,45934029,18009898,11594812,13760171,41894550,13254896,28025170,20007524,13027888}},"employment":[{"organization-name":"Solophase","start-date":date("2010-03-26")}]}
+{"id":11788834,"id-copy":11788834,"alias":"Benny","name":"BennyAgg","user-since":datetime("2011-12-19T14:28:16"),"user-since-copy":datetime("2011-12-19T14:28:16"),"friend-ids":{{6023130,41817759,15338300,40598251,38750529,43646078,9057658}},"employment":[{"organization-name":"Goldcity","start-date":date("2006-09-16")}]}
+{"id":10390954,"id-copy":10390954,"alias":"Lucinda","name":"LucindaWatson","user-since":datetime("2006-11-16T21:20:41"),"user-since-copy":datetime("2006-11-16T21:20:41"),"friend-ids":{{36017573,9298650,16054222,21985420,23378246,30163820,20942039,28917630,20851877,41794807,45887537,39768986,42476881,5070921,29487760,24953551,32065985,16342096,41522555,41923127,34675252,10040601,32604114,23852658}},"employment":[{"organization-name":"Qvohouse","start-date":date("2009-03-14")}]}
+{"id":11626564,"id-copy":11626564,"alias":"Gia","name":"GiaNehling","user-since":datetime("2007-05-04T02:40:35"),"user-since-copy":datetime("2007-05-04T02:40:35"),"friend-ids":{{14435544,22982758,14548448,20359010,43749230,6484290,43513351,3652065,1851524,15523948,1941233,47031188,12649571,42789428,10950757,18325469,24986924,39948729,29738829,268135,32952373,29859037}},"employment":[{"organization-name":"ganjalax","start-date":date("2007-06-13"),"end-date":date("2008-07-06")}]}
+{"id":9311659,"id-copy":9311659,"alias":"Kate","name":"KateBender","user-since":datetime("2007-06-10T05:55:50"),"user-since-copy":datetime("2007-06-10T05:55:50"),"friend-ids":{{27875958,10379355,4286877,26410945,10609943,15960135}},"employment":[{"organization-name":"Hexviafind","start-date":date("2010-01-04")}]}
+{"id":11386210,"id-copy":11386210,"alias":"Dale","name":"DaleGreenwood","user-since":datetime("2007-04-17T19:02:45"),"user-since-copy":datetime("2007-04-17T19:02:45"),"friend-ids":{{3669916}},"employment":[{"organization-name":"sonstreet","start-date":date("2002-09-11")}]}
+{"id":10993267,"id-copy":10993267,"alias":"Esmund","name":"EsmundDunkle","user-since":datetime("2005-11-16T21:18:20"),"user-since-copy":datetime("2005-11-16T21:18:20"),"friend-ids":{{1277480,11393524,32336542,41857626,7807437,25280677,17518254,7723810,18423045,11937236,21507800}},"employment":[{"organization-name":"Canline","start-date":date("2006-12-03"),"end-date":date("2011-11-26")}]}
+{"id":10889389,"id-copy":10889389,"alias":"Roselyn","name":"RoselynLlora","user-since":datetime("2012-03-25T15:21:06"),"user-since-copy":datetime("2012-03-25T15:21:06"),"friend-ids":{{38921827,1378686,22284385,17464785,16302500,47598267,25016712,11151378,16381115,16371401}},"employment":[{"organization-name":"Plexlane","start-date":date("2005-12-02")}]}
+{"id":11441509,"id-copy":11441509,"alias":"Franklyn","name":"FranklynZimmer","user-since":datetime("2012-03-22T13:12:29"),"user-since-copy":datetime("2012-03-22T13:12:29"),"friend-ids":{{12883110,5637339,42139368,25533619,19998291,4231212,40792266,9689761,7591603,29088602,40962884,9432997,29850101,47563888,10384431,30557751,9141240,45176888,40987369,42808497,37891546,8520042,12875368,39706341}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2008-06-09")}]}
+{"id":11209297,"id-copy":11209297,"alias":"Merlin","name":"MerlinLambert","user-since":datetime("2012-07-01T09:30:07"),"user-since-copy":datetime("2012-07-01T09:30:07"),"friend-ids":{{28451212,22119974,1386726,20860479,37160852,38281524,17165711,41076637,19118162}},"employment":[{"organization-name":"Labzatron","start-date":date("2012-06-26"),"end-date":date("2012-06-09")}]}
+{"id":9952339,"id-copy":9952339,"alias":"Dacia","name":"DaciaStaymates","user-since":datetime("2009-09-27T09:55:51"),"user-since-copy":datetime("2009-09-27T09:55:51"),"friend-ids":{{5177020,46967179,24156959,17828131,41565753,1929360,33761670,27544454,9964059,25582191}},"employment":[{"organization-name":"Newfase","start-date":date("2000-10-12"),"end-date":date("2007-01-20")}]}
+{"id":9265747,"id-copy":9265747,"alias":"Nicolas","name":"NicolasPirl","user-since":datetime("2011-11-07T13:52:49"),"user-since-copy":datetime("2011-11-07T13:52:49"),"friend-ids":{{5832017,30839617,27328653,9766355,35973149,21029594,18840511,43035135,44902336,11576374,21756219,23374243,42201568,12860309}},"employment":[{"organization-name":"Canline","start-date":date("2003-11-10"),"end-date":date("2010-03-27")}]}
+{"id":9883165,"id-copy":9883165,"alias":"Dean","name":"DeanKern","user-since":datetime("2005-11-02T13:10:37"),"user-since-copy":datetime("2005-11-02T13:10:37"),"friend-ids":{{33343261,27280204,31345192,723310,11949431,4787422,28427922,11974873,24553234,19067609,12178905,38171944,26832701,47422914,47782561,26391811,28206950,17135029,37069726,40613638,11509775}},"employment":[{"organization-name":"ganjalax","start-date":date("2003-02-16"),"end-date":date("2009-12-16")}]}
+{"id":10045915,"id-copy":10045915,"alias":"Mona","name":"MonaMarshall","user-since":datetime("2005-08-24T06:03:43"),"user-since-copy":datetime("2005-08-24T06:03:43"),"friend-ids":{{34157870,1960568,39038094,2842182,12353591,44464974,45836337,4831806,18179039,21060089,15776264,41865218,5999176,18197780}},"employment":[{"organization-name":"Mathtech","start-date":date("2006-07-28")}]}
+{"id":10278607,"id-copy":10278607,"alias":"Brenden","name":"BrendenLombardi","user-since":datetime("2012-02-13T05:59:40"),"user-since-copy":datetime("2012-02-13T05:59:40"),"friend-ids":{{2820692,43529738,38518064,29672334,24653037,39717291,14213502,23982828,47123006,34213620,5993185,10068793,47512414,40682283,26631237,23442819,9215972,9003752,31259126,8467245,32821220,8582002,42606040}},"employment":[{"organization-name":"Plexlane","start-date":date("2001-06-16"),"end-date":date("2008-09-11")}]}
+{"id":10087876,"id-copy":10087876,"alias":"Carlyle","name":"CarlyleMoberly","user-since":datetime("2009-09-12T03:44:36"),"user-since-copy":datetime("2009-09-12T03:44:36"),"friend-ids":{{22254101,16994379,42146906,28928982}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2012-07-24"),"end-date":date("2012-07-09")}]}
+{"id":10509676,"id-copy":10509676,"alias":"Dinorah","name":"DinorahRopes","user-since":datetime("2009-12-05T06:00:03"),"user-since-copy":datetime("2009-12-05T06:00:03"),"friend-ids":{{13297859,17139775,6500776,46867326,18510471,20417055,39500392,2482383,3361807,14184772,24928547,14390842,40519232,14991589,21242930,24964529,38160860,25523267,4709228,13473948,15850888,30150938,5984402,26343874}},"employment":[{"organization-name":"Sumlane","start-date":date("2010-08-11")}]}
+{"id":11158711,"id-copy":11158711,"alias":"Gwendolen","name":"GwendolenBousum","user-since":datetime("2007-07-06T10:35:24"),"user-since-copy":datetime("2007-07-06T10:35:24"),"friend-ids":{{22558162,31443428,22992355,19452651,23323540,41272500,17328954,37489389,35041092,42476655}},"employment":[{"organization-name":"Striptaxon","start-date":date("2005-05-23")}]}
+{"id":10636498,"id-copy":10636498,"alias":"Grahame","name":"GrahameLeslie","user-since":datetime("2006-01-17T16:17:07"),"user-since-copy":datetime("2006-01-17T16:17:07"),"friend-ids":{{3924169,14543253,19830425,34696361,26630699,47664771}},"employment":[{"organization-name":"Fixdintex","start-date":date("2004-03-25")}]}
+{"id":9125827,"id-copy":9125827,"alias":"Kary","name":"KaryHildyard","user-since":datetime("2006-03-17T23:21:33"),"user-since-copy":datetime("2006-03-17T23:21:33"),"friend-ids":{{5570026}},"employment":[{"organization-name":"Salthex","start-date":date("2007-09-27")}]}
+{"id":11920375,"id-copy":11920375,"alias":"Terance","name":"TeranceSaylor","user-since":datetime("2005-02-09T10:33:47"),"user-since-copy":datetime("2005-02-09T10:33:47"),"friend-ids":{{17869677,39051840,6852335,6153367,1318628,9983745,5401091,32798056,42870494,10337793,43570623,3233493,38297525,43712104,15430099,36703995,25022620,3681464,21499719,33737350,6602331,35391438,47011233}},"employment":[{"organization-name":"Indiex","start-date":date("2005-11-05"),"end-date":date("2011-04-20")}]}
+{"id":9799264,"id-copy":9799264,"alias":"Bradley","name":"BradleyTodd","user-since":datetime("2011-05-18T23:42:33"),"user-since-copy":datetime("2011-05-18T23:42:33"),"friend-ids":{{8836368,35488923,26777243,46550104,9866525,965209}},"employment":[{"organization-name":"Striptaxon","start-date":date("2007-12-22")}]}
+{"id":11287327,"id-copy":11287327,"alias":"Vito","name":"VitoMoffat","user-since":datetime("2008-02-08T03:16:42"),"user-since-copy":datetime("2008-02-08T03:16:42"),"friend-ids":{{36850894,16346016,4072987,36112362,13277841,24976604,20216096,36253616,13624540,39256929,8411929,13545093,27563972,4306316,9819682,21998450,16647991,1987261}},"employment":[{"organization-name":"Icerunin","start-date":date("2001-07-08"),"end-date":date("2005-04-23")}]}
+{"id":9179122,"id-copy":9179122,"alias":"Zach","name":"ZachMilliron","user-since":datetime("2011-07-28T01:09:04"),"user-since-copy":datetime("2011-07-28T01:09:04"),"friend-ids":{{40552138,19204406,46806031,18794200,45071131,40119114,23955279,11126709,37101358,23332998,1172034,41496458,32278235,30949991,148070,6360227,7378339,33611217}},"employment":[{"organization-name":"Freshfix","start-date":date("2000-06-26")}]}
+{"id":9041992,"id-copy":9041992,"alias":"Royston","name":"RoystonBatten","user-since":datetime("2009-06-27T08:09:45"),"user-since-copy":datetime("2009-06-27T08:09:45"),"friend-ids":{{35666317,30439304,35405688,2079220,5996407,40490306,33188983,24455609,4293738,29028817,32566429,10186823}},"employment":[{"organization-name":"Coneflex","start-date":date("2002-04-04"),"end-date":date("2010-06-28")}]}
+{"id":9752227,"id-copy":9752227,"alias":"Audley","name":"AudleyPeters","user-since":datetime("2006-07-27T01:15:35"),"user-since-copy":datetime("2006-07-27T01:15:35"),"friend-ids":{{877448,29611844,2844046,42493473,28216181,353847,44172105,36184409,44010617}},"employment":[{"organization-name":"Hot-tech","start-date":date("2002-12-17")}]}
+{"id":11676574,"id-copy":11676574,"alias":"Isidore","name":"IsidoreCatlay","user-since":datetime("2012-08-26T08:28:08"),"user-since-copy":datetime("2012-08-26T08:28:08"),"friend-ids":{{46189001}},"employment":[{"organization-name":"Codetechno","start-date":date("2001-03-07")}]}
+{"id":10392898,"id-copy":10392898,"alias":"Rodger","name":"RodgerLear","user-since":datetime("2010-03-05T20:39:12"),"user-since-copy":datetime("2010-03-05T20:39:12"),"friend-ids":{{23638180,34355575,28958329,17287883,46069191,4055459,36969931,13059600,6957015,41374655,44549230,1943320,39878243}},"employment":[{"organization-name":"Hexviafind","start-date":date("2002-12-22")}]}
+{"id":11316178,"id-copy":11316178,"alias":"Carlene","name":"CarleneArchibald","user-since":datetime("2007-09-02T16:24:57"),"user-since-copy":datetime("2007-09-02T16:24:57"),"friend-ids":{{45522809,33213012,2265630,27087141,7247502,38659338,33327692,43927391,41809132,4738869,9663680,45809341,38204579,17145650,23991333,9915598,28129675,47406993,37554697}},"employment":[{"organization-name":"Alphadax","start-date":date("2007-12-15"),"end-date":date("2008-06-02")}]}
+{"id":9518128,"id-copy":9518128,"alias":"Jerrie","name":"JerrieFonblanque","user-since":datetime("2008-06-08T02:51:53"),"user-since-copy":datetime("2008-06-08T02:51:53"),"friend-ids":{{41051692,21547608,41749297,21528434,28012731,43579854,9172140,17908381,10276804,12277383,38454166,6950146,11878198,24415804,46218827,33013212,44735001,36395459,38515534,16015324,21085620,20338207}},"employment":[{"organization-name":"U-electrics","start-date":date("2001-01-14")}]}
+{"id":10219465,"id-copy":10219465,"alias":"Ros","name":"RosSurrency","user-since":datetime("2010-04-20T12:07:16"),"user-since-copy":datetime("2010-04-20T12:07:16"),"friend-ids":{{14365151,47786936,41386448,10958072,34068903,28844652,16749120,16920092,7474357,35730197,13732713,26185093,19486844,13720196,7483494,16709415,32998666,31641404,42939361,20750447,44343030,17559252,13810932}},"employment":[{"organization-name":"Tripplelane","start-date":date("2008-12-12"),"end-date":date("2010-05-04")}]}
+{"id":10054327,"id-copy":10054327,"alias":"Poppy","name":"PoppyKellogg","user-since":datetime("2010-03-28T09:43:49"),"user-since-copy":datetime("2010-03-28T09:43:49"),"friend-ids":{{10785684,26545687,942400,18147517,12133643,17848751,40864121,18975370,26159158,42348235,21795276,40155922,35240759}},"employment":[{"organization-name":"Vivaace","start-date":date("2012-03-24")}]}
+{"id":9395638,"id-copy":9395638,"alias":"Toby","name":"TobyThomlinson","user-since":datetime("2012-02-02T02:11:31"),"user-since-copy":datetime("2012-02-02T02:11:31"),"friend-ids":{{39086825,14218540,37526829,46631432,24407673,19484977,3657630}},"employment":[{"organization-name":"tresline","start-date":date("2012-02-26")}]}
+{"id":11252185,"id-copy":11252185,"alias":"Quintin","name":"QuintinMcdonald","user-since":datetime("2010-09-27T08:09:51"),"user-since-copy":datetime("2010-09-27T08:09:51"),"friend-ids":{{17231767,1840658,32389773,31328720,18446903,48007173,40417004,41543048,4774035,43047815,24232919,936390,20744224,39536211,34205950,38429209,399190,38425767,8776604,10360244,28414116,15735235,6431904}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2002-10-04")}]}
+{"id":10065595,"id-copy":10065595,"alias":"Zenobia","name":"ZenobiaHiggens","user-since":datetime("2009-11-06T11:19:47"),"user-since-copy":datetime("2009-11-06T11:19:47"),"friend-ids":{{19623415,12770212,30381171,20436392,33497094,39556081,22592010,44832685,35801007,39682093,26870566,8667589,43790411,24760722,8286108,20709133}},"employment":[{"organization-name":"Sumlane","start-date":date("2001-07-28"),"end-date":date("2004-12-26")}]}
+{"id":9638626,"id-copy":9638626,"alias":"Hisako","name":"HisakoEisaman","user-since":datetime("2008-05-26T23:34:43"),"user-since-copy":datetime("2008-05-26T23:34:43"),"friend-ids":{{17773563,18434504,1082020,40557107,43294701,1982610,8259201,47490886,20044705,35882471,7297053,17276976,38660830,36435103,29511457,3474864,17100964,23978369,6260698,17616437,1617227,18325960,42613056}},"employment":[{"organization-name":"Kanelectrics","start-date":date("2009-07-12")}]}
+{"id":9880603,"id-copy":9880603,"alias":"Davis","name":"DavisRitter","user-since":datetime("2009-12-18T18:55:46"),"user-since-copy":datetime("2009-12-18T18:55:46"),"friend-ids":{{10790833,43529865,23457220,6745186,22333440,39380793,2096806,44121543,29345888,46499780,31896682,35084540,6060378,27402271,18954641}},"employment":[{"organization-name":"Medflex","start-date":date("2002-01-11")}]}
+{"id":9595279,"id-copy":9595279,"alias":"Emmaline","name":"EmmalineSchuth","user-since":datetime("2008-09-12T22:25:17"),"user-since-copy":datetime("2008-09-12T22:25:17"),"friend-ids":{{26784778,6200196,37440596,12250319,21921557,19278082,583040,12012653,21578028,16395818,29088493,29578064,37745574,41998781,22594273,38002130,2166585,7823908,18253304,6162341,40270219,41832701,36455204}},"employment":[{"organization-name":"Villa-tech","start-date":date("2009-02-13")}]}
+{"id":9001816,"id-copy":9001816,"alias":"Concordia","name":"ConcordiaThomlinson","user-since":datetime("2006-04-13T03:30:17"),"user-since-copy":datetime("2006-04-13T03:30:17"),"friend-ids":{{31001079,10620343,29160614,8991085,45471665,865015,11592391,33106281,15448665,29325047,47814022,4562661,11895808,41974900}},"employment":[{"organization-name":"Vaiatech","start-date":date("2002-03-25")}]}
+{"id":10083103,"id-copy":10083103,"alias":"Albertine","name":"AlbertineShick","user-since":datetime("2006-11-10T03:24:02"),"user-since-copy":datetime("2006-11-10T03:24:02"),"friend-ids":{{22979883,41779991,30340160,44852777,43786950,33382165,898482,16427018,1264379,19925419,10166319,12658187,38802346}},"employment":[{"organization-name":"highfax","start-date":date("2010-03-05")}]}
+{"id":9696160,"id-copy":9696160,"alias":"Lawerence","name":"LawerenceLudwig","user-since":datetime("2005-09-04T07:08:01"),"user-since-copy":datetime("2005-09-04T07:08:01"),"friend-ids":{{33125788,14719007,35434564}},"employment":[{"organization-name":"U-ron","start-date":date("2001-02-02")}]}
+{"id":10904125,"id-copy":10904125,"alias":"Jarred","name":"JarredRopes","user-since":datetime("2005-11-09T09:53:06"),"user-since-copy":datetime("2005-11-09T09:53:06"),"friend-ids":{{26810,23763346,5064508,26124598}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2007-12-28"),"end-date":date("2009-04-23")}]}
+{"id":11468158,"id-copy":11468158,"alias":"Pamelia","name":"PameliaShaner","user-since":datetime("2005-07-11T18:28:07"),"user-since-copy":datetime("2005-07-11T18:28:07"),"friend-ids":{{8892753,24751024,7162523,38425260,8752332,23371746,6673241,22278741,46403700}},"employment":[{"organization-name":"Dandamace","start-date":date("2006-02-04")}]}
+{"id":10364356,"id-copy":10364356,"alias":"Katharine","name":"KatharineHoward","user-since":datetime("2012-03-04T04:40:32"),"user-since-copy":datetime("2012-03-04T04:40:32"),"friend-ids":{{38784,9497194,38432548,30160971,16843331,36942612,32507064,41108421,31761239,20202472,37170299,39217222,14201294,46319310}},"employment":[{"organization-name":"Vivaace","start-date":date("2011-11-09"),"end-date":date("2011-07-18")}]}
+{"id":10405423,"id-copy":10405423,"alias":"Pauletta","name":"PaulettaGuess","user-since":datetime("2007-06-11T02:54:36"),"user-since-copy":datetime("2007-06-11T02:54:36"),"friend-ids":{{14845791,24263161,2648994,30766767,10127359,20706390}},"employment":[{"organization-name":"Freshfix","start-date":date("2002-10-27")}]}
+{"id":10721059,"id-copy":10721059,"alias":"Amandine","name":"AmandineRockwell","user-since":datetime("2008-09-24T21:50:39"),"user-since-copy":datetime("2008-09-24T21:50:39"),"friend-ids":{{10360854,15197739,28812340,12172446,9354363,23580760,6364957,20048548}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2003-12-06")}]}
+{"id":9621157,"id-copy":9621157,"alias":"Trixie","name":"TrixieFair","user-since":datetime("2010-12-25T23:36:49"),"user-since-copy":datetime("2010-12-25T23:36:49"),"friend-ids":{{17519006,17545060,27836293,11477603,37895380,23251592,12010503,25406806}},"employment":[{"organization-name":"Ransaofan","start-date":date("2003-09-23")}]}
+{"id":11272591,"id-copy":11272591,"alias":"Caris","name":"CarisCatleay","user-since":datetime("2007-01-27T07:35:12"),"user-since-copy":datetime("2007-01-27T07:35:12"),"friend-ids":{{26014944}},"employment":[{"organization-name":"Qvohouse","start-date":date("2012-07-15"),"end-date":date("2012-07-01")}]}
+{"id":11559613,"id-copy":11559613,"alias":"Mick","name":"MickWilkinson","user-since":datetime("2005-12-23T15:11:33"),"user-since-copy":datetime("2005-12-23T15:11:33"),"friend-ids":{{4641355}},"employment":[{"organization-name":"Canline","start-date":date("2000-06-03")}]}
+{"id":10073632,"id-copy":10073632,"alias":"Hadley","name":"HadleyPainter","user-since":datetime("2010-08-18T16:57:45"),"user-since-copy":datetime("2010-08-18T16:57:45"),"friend-ids":{{35310707,40074121,28614727,29388510,29966750,45475518,5989395,9892960,7137969,5530675,2278234,9571067,29644726,30689189,41083149}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2004-06-13"),"end-date":date("2004-11-28")}]}
+{"id":9978190,"id-copy":9978190,"alias":"Tatianna","name":"TatiannaSchmidt","user-since":datetime("2012-07-05T14:37:56"),"user-since-copy":datetime("2012-07-05T14:37:56"),"friend-ids":{{15128198}},"employment":[{"organization-name":"silfind","start-date":date("2008-11-17")}]}
+{"id":10227844,"id-copy":10227844,"alias":"Simon","name":"SimonCoates","user-since":datetime("2008-09-18T06:23:35"),"user-since-copy":datetime("2008-09-18T06:23:35"),"friend-ids":{{5847048,15554997,1367924,17223026,31605674,38148868,15521228,37540102,4103855,39184726,26130198,43081715,35929397,28963043,10703925}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2011-07-02")}]}
+{"id":10668283,"id-copy":10668283,"alias":"Dorian","name":"DorianTomlinson","user-since":datetime("2008-06-22T00:01:46"),"user-since-copy":datetime("2008-06-22T00:01:46"),"friend-ids":{{}},"employment":[{"organization-name":"Lexitechno","start-date":date("2004-12-17")}]}
+{"id":9676201,"id-copy":9676201,"alias":"Jessica","name":"JessicaBeals","user-since":datetime("2006-12-02T17:13:07"),"user-since-copy":datetime("2006-12-02T17:13:07"),"friend-ids":{{40180348,5499689,43937013,12294744,47607871,15173594,19403387,30591667,1488569,11862843,26230465,15334606,4397778,8140277,39859715,25854759,7216524,41695061,43036500,15618315,4503056,23790965,14510949,34347866}},"employment":[{"organization-name":"Doncare","start-date":date("2011-05-15"),"end-date":date("2011-10-27")}]}
+{"id":10860286,"id-copy":10860286,"alias":"Albert","name":"AlbertMills","user-since":datetime("2005-01-04T04:39:49"),"user-since-copy":datetime("2005-01-04T04:39:49"),"friend-ids":{{45171802,36246654,30029601,40155304,4876814,275363,46427463,5698619,34383185,47844520,45026162,33852471,36744791,40565586,47142152,42828565}},"employment":[{"organization-name":"subtam","start-date":date("2012-02-20"),"end-date":date("2012-03-21")}]}
+{"id":9693988,"id-copy":9693988,"alias":"Geordie","name":"GeordieBunten","user-since":datetime("2006-08-03T15:00:25"),"user-since-copy":datetime("2006-08-03T15:00:25"),"friend-ids":{{31987089,15556815,3656365,35713356,9573642,38459850,44400137,44882118,44921684,47393814,7869122,35085016,43725704,17602789,9966406,20936803,26425879,41666932}},"employment":[{"organization-name":"Xx-drill","start-date":date("2007-01-20")}]}
+{"id":9979750,"id-copy":9979750,"alias":"Reginald","name":"ReginaldAltman","user-since":datetime("2007-04-04T08:51:58"),"user-since-copy":datetime("2007-04-04T08:51:58"),"friend-ids":{{2988287}},"employment":[{"organization-name":"Transhigh","start-date":date("2002-01-28")}]}
+{"id":10901332,"id-copy":10901332,"alias":"Caelie","name":"CaelieShafer","user-since":datetime("2011-09-24T05:08:05"),"user-since-copy":datetime("2011-09-24T05:08:05"),"friend-ids":{{40761096,31796928,1066172,21271172,41179382,46260705,9287042,37605846,18083603,23469027,45497916,10102434,724885,31794816,44125905,46373183,28321712}},"employment":[{"organization-name":"Tranzap","start-date":date("2012-07-04")}]}
+{"id":9234529,"id-copy":9234529,"alias":"Xavior","name":"XaviorBarnes","user-since":datetime("2010-08-26T12:06:44"),"user-since-copy":datetime("2010-08-26T12:06:44"),"friend-ids":{{19552290,24018104,43285028,33954718,18084047,18675363,17369450,36533551,46779811,46943171,17609996,14171942,10468121,33831228,9905114,11839935,41387228}},"employment":[{"organization-name":"Y-geohex","start-date":date("2007-12-24")}]}
+{"id":9449881,"id-copy":9449881,"alias":"Veola","name":"VeolaSchaeffer","user-since":datetime("2005-06-15T04:27:55"),"user-since-copy":datetime("2005-06-15T04:27:55"),"friend-ids":{{15932585,16875491,977001,15650783,30629770,9735829,35435062,2023808,21909452,29327288,24004438,41780113,10546865,17514287,16690971,23762008,21853049,12673064,35992661,30579445,21341455,2338670}},"employment":[{"organization-name":"Y-geohex","start-date":date("2001-09-07")}]}
+{"id":10090042,"id-copy":10090042,"alias":"Gaye","name":"GayeHayhurst","user-since":datetime("2006-09-23T14:26:31"),"user-since-copy":datetime("2006-09-23T14:26:31"),"friend-ids":{{41099035,16443590,9899624,2459064,25428448,1420220,1487058,13700561,11008052,36459693,45632468,30351729,33053870,26372759,10801940,37166367}},"employment":[{"organization-name":"Vaiatech","start-date":date("2005-07-15"),"end-date":date("2010-05-04")}]}
+{"id":9133714,"id-copy":9133714,"alias":"Wil","name":"WilDale","user-since":datetime("2009-12-04T18:40:04"),"user-since-copy":datetime("2009-12-04T18:40:04"),"friend-ids":{{40400811,26528322}},"employment":[{"organization-name":"Fix-touch","start-date":date("2005-10-08"),"end-date":date("2007-03-23")}]}
+{"id":11507149,"id-copy":11507149,"alias":"Kendal","name":"KendalCourtney","user-since":datetime("2006-06-22T04:28:09"),"user-since-copy":datetime("2006-06-22T04:28:09"),"friend-ids":{{9084267,26163683,15271756,4229254,5439809,23992890,23144677,26584955,29430424,15196312,19993838,3665259,15861241,15197583,15693177}},"employment":[{"organization-name":"Streettax","start-date":date("2010-08-06"),"end-date":date("2011-04-21")}]}
+{"id":11116594,"id-copy":11116594,"alias":"Norwood","name":"NorwoodErrett","user-since":datetime("2008-10-04T16:36:27"),"user-since-copy":datetime("2008-10-04T16:36:27"),"friend-ids":{{30996403,30788997,22512789,35425088,12096858,21391496,41281428,15854003,47041757,31205204,36849089,43015828,27098245,46735331,9520980,34482257,36898055,8962397}},"employment":[{"organization-name":"Ronholdings","start-date":date("2002-05-20")}]}
+{"id":11097556,"id-copy":11097556,"alias":"Tia","name":"TiaHair","user-since":datetime("2010-10-28T01:21:36"),"user-since-copy":datetime("2010-10-28T01:21:36"),"friend-ids":{{19746022,42650092,45679457,43873545,5490025,42900988,32855768,20717716,15007194,23035301,24322095,27796211,27751858,4726224,5570083,18421959,28424121,22311092,13781420,18215783,19934706,18408890,24792739,4022527}},"employment":[{"organization-name":"Trustbam","start-date":date("2003-04-03")}]}
+{"id":10085446,"id-copy":10085446,"alias":"Merla","name":"MerlaWhitehead","user-since":datetime("2006-12-08T11:13:30"),"user-since-copy":datetime("2006-12-08T11:13:30"),"friend-ids":{{44039547}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2002-03-16"),"end-date":date("2009-04-16")}]}
+{"id":11366056,"id-copy":11366056,"alias":"Devin","name":"DevinUlery","user-since":datetime("2011-05-03T13:27:51"),"user-since-copy":datetime("2011-05-03T13:27:51"),"friend-ids":{{25443767,42385070,31515075,31340661,25371541,34378389,40381786,23698797,40141450,12814851,41414503,39733660,27910438,44106204,18806338,37909692,12502759,4270087,5110443,14347603,19313129,8826229}},"employment":[{"organization-name":"sonstreet","start-date":date("2001-12-15")}]}
+{"id":10252147,"id-copy":10252147,"alias":"Concha","name":"ConchaMckinnon","user-since":datetime("2009-12-21T03:27:35"),"user-since-copy":datetime("2009-12-21T03:27:35"),"friend-ids":{{8837048,7758233,2108777,31062874,34698247,33766563,10653492,25103733,24629375,38758275,37539109,47252638,41559516,41883197,9608881,26501553,39435548,43307321,46890131,29908109}},"employment":[{"organization-name":"jaydax","start-date":date("2011-05-09")}]}
+{"id":10749553,"id-copy":10749553,"alias":"Rolland","name":"RollandMunshower","user-since":datetime("2005-12-26T19:26:32"),"user-since-copy":datetime("2005-12-26T19:26:32"),"friend-ids":{{27080985,4355429,17027260,30203290,37292858,1935550,467329,24265915,4926329,28586308,27299677,25356918,14171255,319307,15014794}},"employment":[{"organization-name":"Lexicone","start-date":date("2011-04-21")}]}
+{"id":9856990,"id-copy":9856990,"alias":"Claud","name":"ClaudBaird","user-since":datetime("2006-10-10T11:48:09"),"user-since-copy":datetime("2006-10-10T11:48:09"),"friend-ids":{{41756695,15842897,29797715,13771892,21179308,42974840,22223660,35004748,35597685,45300254,31116834,42699991,9704157,23181215,14806554,8198556,16256974,16360634,34736641}},"employment":[{"organization-name":"Canline","start-date":date("2008-07-23")}]}
+{"id":9219955,"id-copy":9219955,"alias":"Audrey","name":"AudreyOmara","user-since":datetime("2011-06-04T15:31:25"),"user-since-copy":datetime("2011-06-04T15:31:25"),"friend-ids":{{28209595,29907721,18295175,18631813,3380755,20244076,43026452,42394327,10914853,27224999}},"employment":[{"organization-name":"overtech","start-date":date("2003-03-24")}]}
+{"id":10126408,"id-copy":10126408,"alias":"Pen","name":"PenFleming","user-since":datetime("2005-11-11T08:50:34"),"user-since-copy":datetime("2005-11-11T08:50:34"),"friend-ids":{{38072630,45021886,23988042,41084533,4743969,7223979,19120365,44219284,4691449,21072839,32536521,36335527,47376347,16882811,43140173,7610811,28217191,25488874,27968660,13102347,40169395,25952056,17249838,30971677}},"employment":[{"organization-name":"Plexlane","start-date":date("2011-10-04"),"end-date":date("2011-01-10")}]}
+{"id":11380807,"id-copy":11380807,"alias":"Mckinley","name":"MckinleyGeyer","user-since":datetime("2008-02-17T13:01:21"),"user-since-copy":datetime("2008-02-17T13:01:21"),"friend-ids":{{16655526,20048717,15998744,39702027,28153175,40825599,38372618}},"employment":[{"organization-name":"Kanelectrics","start-date":date("2010-11-26")}]}
+{"id":11774587,"id-copy":11774587,"alias":"Shari","name":"ShariMortland","user-since":datetime("2012-07-21T10:15:22"),"user-since-copy":datetime("2012-07-21T10:15:22"),"friend-ids":{{17661326,29399532,38328734,38063295,46008807,29873254,4407085,27903240}},"employment":[{"organization-name":"Statcode","start-date":date("2005-05-18")}]}
+{"id":10025086,"id-copy":10025086,"alias":"Peggy","name":"PeggyOlphert","user-since":datetime("2009-06-24T16:14:48"),"user-since-copy":datetime("2009-06-24T16:14:48"),"friend-ids":{{13659719,46045788,35841713,32392118,24785179,45483286,47287227,42691471,7471992,47671331,25747076,2368606,34452743,14570607,31436760,36423303,31381129,29414651,10005587,14082638,13311890,11592210,1585557}},"employment":[{"organization-name":"Dandamace","start-date":date("2008-07-20")}]}
+{"id":11327731,"id-copy":11327731,"alias":"Duncan","name":"DuncanPennington","user-since":datetime("2007-09-08T05:38:28"),"user-since-copy":datetime("2007-09-08T05:38:28"),"friend-ids":{{7591038,8046115,16606742,39494564,32760725,39036737,9937167,38968828,32536611}},"employment":[{"organization-name":"linedexon","start-date":date("2003-12-06")}]}
+{"id":11610913,"id-copy":11610913,"alias":"Vic","name":"VicDiegel","user-since":datetime("2008-08-03T21:05:21"),"user-since-copy":datetime("2008-08-03T21:05:21"),"friend-ids":{{15275871,8304749,7803583,45134147,36058489,7180792,2104280,4322584,39304177,43050196,32955811,4161448,3187410,47263593}},"employment":[{"organization-name":"Xx-drill","start-date":date("2001-03-11")}]}
+{"id":10577128,"id-copy":10577128,"alias":"Charnette","name":"CharnettePyle","user-since":datetime("2008-08-20T21:25:22"),"user-since-copy":datetime("2008-08-20T21:25:22"),"friend-ids":{{30078840,16315930,12006652,31984600,12053254,41773411,43318427,21592935,40739515,30608076,21922300,5687640}},"employment":[{"organization-name":"Alphadax","start-date":date("2001-11-25"),"end-date":date("2002-08-12")}]}
+{"id":10508467,"id-copy":10508467,"alias":"Quincey","name":"QuinceyKettlewell","user-since":datetime("2009-11-08T14:09:57"),"user-since-copy":datetime("2009-11-08T14:09:57"),"friend-ids":{{16037923,33757766,22829568,34589661,10645853,43124745,41785968,27704416,42381402,11993654,31993782,37761743,15571469,33326934,22719288,18321279,19252211,42927515,22390312,37655021,37511969,47740024,1015876}},"employment":[{"organization-name":"Sumlane","start-date":date("2009-02-24")}]}
+{"id":9820681,"id-copy":9820681,"alias":"Caitlin","name":"CaitlinWolfe","user-since":datetime("2012-05-23T07:59:39"),"user-since-copy":datetime("2012-05-23T07:59:39"),"friend-ids":{{22005473,7664709,22913945,16078115,11724028,45958589,33357270,6935384,2696233,28938665,37992833,11929142,16203505,20365802}},"employment":[{"organization-name":"Fix-touch","start-date":date("2010-06-18")}]}
+{"id":10623790,"id-copy":10623790,"alias":"Leon","name":"LeonSouthern","user-since":datetime("2006-08-26T12:47:17"),"user-since-copy":datetime("2006-08-26T12:47:17"),"friend-ids":{{15974929,10054172,9775689,22060162,41777649,13548836,10842789,45455670,32027368,45268626,40570545,18214851,47559589,38267347,41101925,45749689,29277572,47828706,45708476,33769625}},"employment":[{"organization-name":"Redelectronics","start-date":date("2012-06-22"),"end-date":date("2012-06-05")}]}
+{"id":11226055,"id-copy":11226055,"alias":"Tony","name":"TonyBowman","user-since":datetime("2011-06-27T19:37:38"),"user-since-copy":datetime("2011-06-27T19:37:38"),"friend-ids":{{38143523,845148,17273955,5476646,28032520,29082922,26004648,7037738,34413190,22897549,19873990,22338498,10902206,43469888,21968875,5127825,11962760,43764181,20623302,23901531,3402018,15386752,30847912,205201}},"employment":[{"organization-name":"Greencare","start-date":date("2004-09-13"),"end-date":date("2011-01-10")}]}
+{"id":11951800,"id-copy":11951800,"alias":"Camron","name":"CamronBrooks","user-since":datetime("2006-03-05T19:32:03"),"user-since-copy":datetime("2006-03-05T19:32:03"),"friend-ids":{{39430755,45789857,5352132,34490450,39117503,2233039,16387184}},"employment":[{"organization-name":"Ransaofan","start-date":date("2006-12-26"),"end-date":date("2007-11-16")}]}
+{"id":9292738,"id-copy":9292738,"alias":"Walter","name":"WalterWain","user-since":datetime("2012-05-03T10:41:22"),"user-since-copy":datetime("2012-05-03T10:41:22"),"friend-ids":{{1834068,38777276,43381631,32380769,23994313,37459142,21015234,23788270,33191448,31111521,21788604,39349512,20638072,17300228,4712935,36205876,27740958,27236154}},"employment":[{"organization-name":"Zununoing","start-date":date("2011-03-28")}]}
+{"id":9636802,"id-copy":9636802,"alias":"Gage","name":"GageHair","user-since":datetime("2011-01-23T22:31:49"),"user-since-copy":datetime("2011-01-23T22:31:49"),"friend-ids":{{46795684,38195763,25882078,28871879,5178144,17683475,43441471,5427133,13936915,2608474,9513798,31041524,557454,22452168,12948004,16835098,1151241,37188687}},"employment":[{"organization-name":"Sublamdox","start-date":date("2000-05-02"),"end-date":date("2010-02-13")}]}
+{"id":11708152,"id-copy":11708152,"alias":"Gil","name":"GilElsas","user-since":datetime("2009-04-08T15:40:59"),"user-since-copy":datetime("2009-04-08T15:40:59"),"friend-ids":{{14661698,22657473,28892770,39654430,46338819,44974094,38564659,24819725,21550883,37711934,37285158,20050610,19163447,10974750,47513067,43771947,23633824}},"employment":[{"organization-name":"Unijobam","start-date":date("2002-09-21"),"end-date":date("2011-03-11")}]}
+{"id":10472248,"id-copy":10472248,"alias":"Harry","name":"HarryDugmore","user-since":datetime("2012-02-18T05:46:12"),"user-since-copy":datetime("2012-02-18T05:46:12"),"friend-ids":{{30193978,30762534,24660208,29628319,30687391,39795396,33525293,23739628,28969085,30275276,3497701,17091988,15259527,25164171,34052417,4318314,1876063,29984074,3421436,16610126}},"employment":[{"organization-name":"Solophase","start-date":date("2012-01-19"),"end-date":date("2012-01-02")}]}
+{"id":10108534,"id-copy":10108534,"alias":"Moriah","name":"MoriahMitchell","user-since":datetime("2005-11-13T21:32:41"),"user-since-copy":datetime("2005-11-13T21:32:41"),"friend-ids":{{30372632}},"employment":[{"organization-name":"whitestreet","start-date":date("2012-05-07")}]}
+{"id":9545626,"id-copy":9545626,"alias":"Russell","name":"RussellKeilbach","user-since":datetime("2010-05-20T15:10:25"),"user-since-copy":datetime("2010-05-20T15:10:25"),"friend-ids":{{40592323,28819303}},"employment":[{"organization-name":"Vaiatech","start-date":date("2003-04-18")}]}
+{"id":11424097,"id-copy":11424097,"alias":"Vernie","name":"VernieWynter","user-since":datetime("2009-02-15T02:35:16"),"user-since-copy":datetime("2009-02-15T02:35:16"),"friend-ids":{{41874621,26330221,38930134,39892396,42859035,8165423,36128938,5692990,28144348,40741492}},"employment":[{"organization-name":"Tranzap","start-date":date("2002-04-06")}]}
+{"id":11087224,"id-copy":11087224,"alias":"Zola","name":"ZolaKnisely","user-since":datetime("2005-11-18T05:30:00"),"user-since-copy":datetime("2005-11-18T05:30:00"),"friend-ids":{{6324130,38065951,14950455,27869167,32957819,11157656,10411400,18072233,35246039,35345326,23217009,13495953,18987122}},"employment":[{"organization-name":"silfind","start-date":date("2003-05-04")}]}
+{"id":9629923,"id-copy":9629923,"alias":"Adria","name":"AdriaBoyer","user-since":datetime("2005-08-12T16:31:38"),"user-since-copy":datetime("2005-08-12T16:31:38"),"friend-ids":{{43812176,1271309,1412045,18793840,40264072,41525831,25536841,46110606,40440782,37228709,37745315,19025404,13458371,32475836,30506186,6860193,44650222,5924034}},"employment":[{"organization-name":"Doncare","start-date":date("2012-03-08")}]}
+{"id":11633326,"id-copy":11633326,"alias":"Jodi","name":"JodiBrindle","user-since":datetime("2009-01-02T19:57:58"),"user-since-copy":datetime("2009-01-02T19:57:58"),"friend-ids":{{5287281,24414393,31942570,45025515,35679462,45244705,4931287,11590610,39846242,14999029,38735562,6275771,33435194}},"employment":[{"organization-name":"Vivaace","start-date":date("2009-11-25")}]}
+{"id":9904822,"id-copy":9904822,"alias":"Judith","name":"JudithChristman","user-since":datetime("2005-05-19T14:43:44"),"user-since-copy":datetime("2005-05-19T14:43:44"),"friend-ids":{{}},"employment":[{"organization-name":"highfax","start-date":date("2002-05-06")}]}
+{"id":9556570,"id-copy":9556570,"alias":"Kassandra","name":"KassandraKern","user-since":datetime("2010-12-03T15:29:12"),"user-since-copy":datetime("2010-12-03T15:29:12"),"friend-ids":{{35944118,3024691,43927521,44121317,29834404,18626717,47095811,38438153,30557309,37143411,41634172,23338449,30455300,12009022,26366377,36381324,25084236,36521163,20063914,11419154,40243010,9336807,3544397,20455720}},"employment":[{"organization-name":"Zununoing","start-date":date("2005-02-12")}]}
+{"id":9367306,"id-copy":9367306,"alias":"Jacinth","name":"JacinthBynum","user-since":datetime("2012-03-08T11:26:04"),"user-since-copy":datetime("2012-03-08T11:26:04"),"friend-ids":{{35048012,42620612,39526901,12673410,16363143,45509270,47714729,47902094,12551745,45510597,31513255,2848992,16088751,1953590,32956014,38607548,15982103,31161780,7331812,44977526,15022020,19905573}},"employment":[{"organization-name":"ganjalax","start-date":date("2003-03-24")}]}
+{"id":9634393,"id-copy":9634393,"alias":"Burt","name":"BurtPearson","user-since":datetime("2007-11-01T14:25:29"),"user-since-copy":datetime("2007-11-01T14:25:29"),"friend-ids":{{26065414,8710639,22639162,23787625,24443211,42598742,45171006,38246985,25125478,23071168,22455706,24720860,34917747,24262081,2259812,14262605,37533604}},"employment":[{"organization-name":"Dandamace","start-date":date("2001-05-07")}]}
+{"id":10002907,"id-copy":10002907,"alias":"Maegan","name":"MaeganErschoff","user-since":datetime("2011-10-15T18:08:56"),"user-since-copy":datetime("2011-10-15T18:08:56"),"friend-ids":{{}},"employment":[{"organization-name":"Fax-fax","start-date":date("2009-02-06"),"end-date":date("2011-05-20")}]}
+{"id":10190329,"id-copy":10190329,"alias":"Rachyl","name":"RachylAdams","user-since":datetime("2005-08-25T14:09:48"),"user-since-copy":datetime("2005-08-25T14:09:48"),"friend-ids":{{}},"employment":[{"organization-name":"Fax-fax","start-date":date("2002-11-17")}]}
+{"id":10118077,"id-copy":10118077,"alias":"Elizbeth","name":"ElizbethPfeifer","user-since":datetime("2011-09-08T11:58:48"),"user-since-copy":datetime("2011-09-08T11:58:48"),"friend-ids":{{18001251,40309720,10119557,37766102,22202316,2805709,693628,5524288,21415560,45687644,23912525,25418741,22816155,26787291,30518473,27701649}},"employment":[{"organization-name":"Sumlane","start-date":date("2002-03-15"),"end-date":date("2004-11-03")}]}
+{"id":10913971,"id-copy":10913971,"alias":"Marylyn","name":"MarylynBuehler","user-since":datetime("2008-03-02T11:14:18"),"user-since-copy":datetime("2008-03-02T11:14:18"),"friend-ids":{{36555710,21041383,37895483,11392039,5195346,12022072,5206222,37834919,434970,4441054,39212196,12773393}},"employment":[{"organization-name":"Ontohothex","start-date":date("2000-01-11")}]}
+{"id":10951918,"id-copy":10951918,"alias":"Doran","name":"DoranBell","user-since":datetime("2005-08-22T14:07:50"),"user-since-copy":datetime("2005-08-22T14:07:50"),"friend-ids":{{6952033,22223086,5858716,35128893,22115927,5821006,16264772,4151991,40384467,19801357,42871024,46855275,35241988,17208259,47420533,25182232,14247140,19664015,33132502,47813026,12819081,29321093,42851957,30756972}},"employment":[{"organization-name":"Unijobam","start-date":date("2012-01-22")}]}
+{"id":11598403,"id-copy":11598403,"alias":"Jo","name":"JoCattley","user-since":datetime("2008-01-04T03:33:03"),"user-since-copy":datetime("2008-01-04T03:33:03"),"friend-ids":{{28948698,9851844,31708351,28418023,33052184,24995451,2840550,19426008,3790086}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2006-09-15")}]}
+{"id":9056494,"id-copy":9056494,"alias":"Alvena","name":"AlvenaPearsall","user-since":datetime("2005-08-09T08:50:25"),"user-since-copy":datetime("2005-08-09T08:50:25"),"friend-ids":{{26263956,80589,37669623,32875186,42026139,22169384,47224581,25632957,28392334,42393204,15028714,554526}},"employment":[{"organization-name":"Salthex","start-date":date("2005-02-19")}]}
+{"id":11390830,"id-copy":11390830,"alias":"Luciano","name":"LucianoHooker","user-since":datetime("2006-08-16T08:17:56"),"user-since-copy":datetime("2006-08-16T08:17:56"),"friend-ids":{{42206490,5533465,32480435,18058343}},"employment":[{"organization-name":"Y-geohex","start-date":date("2004-02-19")}]}
+{"id":11109553,"id-copy":11109553,"alias":"Walker","name":"WalkerDrennan","user-since":datetime("2007-05-03T02:10:46"),"user-since-copy":datetime("2007-05-03T02:10:46"),"friend-ids":{{38288636,35385410,24803705,31461936,34309407}},"employment":[{"organization-name":"jaydax","start-date":date("2010-05-20")}]}
+{"id":9698980,"id-copy":9698980,"alias":"Leland","name":"LelandReiss","user-since":datetime("2012-05-23T04:40:29"),"user-since-copy":datetime("2012-05-23T04:40:29"),"friend-ids":{{7623016,12672253,42612513,44457047,46981337,1098470,23122899,15019916,45345438,30272843,43546610}},"employment":[{"organization-name":"U-ron","start-date":date("2001-11-27")}]}
+{"id":10299298,"id-copy":10299298,"alias":"Belinda","name":"BelindaRockwell","user-since":datetime("2005-03-08T07:13:05"),"user-since-copy":datetime("2005-03-08T07:13:05"),"friend-ids":{{31301282,34653696,23868758}},"employment":[{"organization-name":"sonstreet","start-date":date("2008-08-19")}]}
+{"id":11224090,"id-copy":11224090,"alias":"Alayna","name":"AlaynaHay","user-since":datetime("2008-12-27T11:44:03"),"user-since-copy":datetime("2008-12-27T11:44:03"),"friend-ids":{{9220004,31827642,27616881,26175415,43152043,36272681,669731,40783516,31718359,47123044,24487696,31178381,39602057,2619975,27562896,29215321,35104306,909466,18897009,35295634}},"employment":[{"organization-name":"geomedia","start-date":date("2003-02-01"),"end-date":date("2007-02-07")}]}
+{"id":10938328,"id-copy":10938328,"alias":"Tyrese","name":"TyreseStainforth","user-since":datetime("2011-03-03T04:21:04"),"user-since-copy":datetime("2011-03-03T04:21:04"),"friend-ids":{{33557445,27981614,25595450,31820772,42028444,31389097,16332592,3555278,45113070,5198333}},"employment":[{"organization-name":"Labzatron","start-date":date("2000-06-04")}]}
+{"id":10967305,"id-copy":10967305,"alias":"Harrietta","name":"HarriettaClewett","user-since":datetime("2008-05-11T02:34:28"),"user-since-copy":datetime("2008-05-11T02:34:28"),"friend-ids":{{3346670,25522849,46919524,22773543,8985252,43521041,14951485,45977993,21285106,17023357,615364,23079537,23459313,31663735,24201883,39321873,47183802,26870642,34447310,4848880,17078809,14119447,39460378}},"employment":[{"organization-name":"Technohow","start-date":date("2012-05-12"),"end-date":date("2012-06-25")}]}
+{"id":9840013,"id-copy":9840013,"alias":"Inger","name":"IngerRuhl","user-since":datetime("2009-05-27T20:14:42"),"user-since-copy":datetime("2009-05-27T20:14:42"),"friend-ids":{{36044692}},"employment":[{"organization-name":"Newcom","start-date":date("2010-09-15")}]}
+{"id":10202302,"id-copy":10202302,"alias":"Camila","name":"CamilaKelley","user-since":datetime("2010-04-17T06:57:52"),"user-since-copy":datetime("2010-04-17T06:57:52"),"friend-ids":{{21392718,41703679,41044232,47307848,13912958,45329595,33360889,24572594,23726460,9181899,42227287,26565775,12665691,12244453,26966326,3189268,41340076,33904406,38048631,22870005}},"employment":[{"organization-name":"Sumlane","start-date":date("2000-10-09")}]}
+{"id":11341747,"id-copy":11341747,"alias":"Margaux","name":"MargauxBynum","user-since":datetime("2009-01-16T19:54:27"),"user-since-copy":datetime("2009-01-16T19:54:27"),"friend-ids":{{27056110,1770280,17190314,18164827,32684926,32410281,27173037,16864868,4664026,31170366,4296651}},"employment":[{"organization-name":"Whitemedia","start-date":date("2008-08-20")}]}
+{"id":11405905,"id-copy":11405905,"alias":"Maria","name":"MariaMoore","user-since":datetime("2010-05-22T22:23:16"),"user-since-copy":datetime("2010-05-22T22:23:16"),"friend-ids":{{31883861,37245457,28570944,34781997,8502652,44653970,20757487,13575261,13950179,14347829,35701908,35781889,12226908,35939258,5106463,43910072,10696743,21876393,2309465,1889615}},"employment":[{"organization-name":"Doublezone","start-date":date("2008-03-27")}]}
+{"id":10257028,"id-copy":10257028,"alias":"Gary","name":"GaryThompson","user-since":datetime("2009-01-23T04:15:30"),"user-since-copy":datetime("2009-01-23T04:15:30"),"friend-ids":{{46006273,33435458,40976127,42353737,37166855,14882549,27357892,31126471,38151307,38721200}},"employment":[{"organization-name":"Coneflex","start-date":date("2008-05-03"),"end-date":date("2011-09-08")}]}
+{"id":10907953,"id-copy":10907953,"alias":"Wymond","name":"WymondSnyder","user-since":datetime("2006-02-25T03:33:22"),"user-since-copy":datetime("2006-02-25T03:33:22"),"friend-ids":{{16280602,26846293,39235173,4686537,30457440,23649561,34348317,28099021,1622222,24073647,4742953,14925763,17026705,46257859,22592244}},"employment":[{"organization-name":"Fixdintex","start-date":date("2012-07-22")}]}
+{"id":11626678,"id-copy":11626678,"alias":"Reed","name":"ReedHaile","user-since":datetime("2011-05-28T09:52:04"),"user-since-copy":datetime("2011-05-28T09:52:04"),"friend-ids":{{38955792,36648350,7510300,36168809,41493759,45265187,1653351,44881482,44038304}},"employment":[{"organization-name":"Quoline","start-date":date("2012-03-08"),"end-date":date("2012-05-08")}]}
+{"id":9765517,"id-copy":9765517,"alias":"Alexia","name":"AlexiaTownsend","user-since":datetime("2006-02-23T13:26:33"),"user-since-copy":datetime("2006-02-23T13:26:33"),"friend-ids":{{39892441,43413199,45070224,46877180,24247279,26450737,29111107,46768934,11833332,25913646,43063781}},"employment":[{"organization-name":"Whitemedia","start-date":date("2003-02-08")}]}
+{"id":10931647,"id-copy":10931647,"alias":"Bertina","name":"BertinaStraub","user-since":datetime("2011-05-25T19:21:43"),"user-since-copy":datetime("2011-05-25T19:21:43"),"friend-ids":{{12208030,43810737,43870253,20720324,7601394,22266404,21210273,10076577,25757258,1909792,26189079,37799329,24923233,31687015,37580896,44906728,46928405,10679805,14520239,1690125,37459202,36684838,30982356}},"employment":[{"organization-name":"Y-geohex","start-date":date("2011-02-17"),"end-date":date("2011-06-20")}]}
+{"id":9102208,"id-copy":9102208,"alias":"Lottie","name":"LottieReddish","user-since":datetime("2007-05-22T00:42:45"),"user-since-copy":datetime("2007-05-22T00:42:45"),"friend-ids":{{45227463,22488433,39033954,40377121,17357169,8890953,1623690,11657739,489001,26227491,29459012,39985553,3584598,6381312,22457740,43317482,40035088,29397671,18293877,6788834,44860241}},"employment":[{"organization-name":"Mathtech","start-date":date("2009-04-08")}]}
+{"id":10066711,"id-copy":10066711,"alias":"Nichelle","name":"NichelleErschoff","user-since":datetime("2009-11-10T21:17:50"),"user-since-copy":datetime("2009-11-10T21:17:50"),"friend-ids":{{19024226,24428716,24428406,10686682,46410623,45809403,33158503}},"employment":[{"organization-name":"Vaiatech","start-date":date("2004-06-21"),"end-date":date("2005-08-01")}]}
+{"id":9179413,"id-copy":9179413,"alias":"Benton","name":"BentonMorland","user-since":datetime("2006-02-08T13:43:03"),"user-since-copy":datetime("2006-02-08T13:43:03"),"friend-ids":{{25229017,22411534,46862190,17238544,10875646,19572187,9889710,23940269,24489112,7997331,8866147,29705622,35336434,14756488,40059408,32606759,37546068,24168033,20761302,45465986,27519909,23920570,3984052,38799668}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2008-12-05")}]}
+{"id":11626156,"id-copy":11626156,"alias":"Laurine","name":"LaurineBastion","user-since":datetime("2012-05-14T21:34:43"),"user-since-copy":datetime("2012-05-14T21:34:43"),"friend-ids":{{13978691,24432513,41105156,4981880}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2000-03-09")}]}
+{"id":9286279,"id-copy":9286279,"alias":"Barnaby","name":"BarnabyAckerley","user-since":datetime("2006-09-15T01:56:34"),"user-since-copy":datetime("2006-09-15T01:56:34"),"friend-ids":{{21236050,22647474,18898492,22530993,4332450,38947319,25882415,47187086,5810354,18396369,44918707,9732196,14821426,148735}},"employment":[{"organization-name":"itlab","start-date":date("2011-02-10")}]}
+{"id":9203731,"id-copy":9203731,"alias":"Phoebe","name":"PhoebeCoates","user-since":datetime("2008-04-27T01:42:34"),"user-since-copy":datetime("2008-04-27T01:42:34"),"friend-ids":{{25611465,519838,22814080,46015954,7805914,12757618,36785422,25727822,32042543}},"employment":[{"organization-name":"over-it","start-date":date("2003-11-11"),"end-date":date("2005-08-19")}]}
+{"id":10501429,"id-copy":10501429,"alias":"Danielle","name":"DanielleYoung","user-since":datetime("2010-04-24T05:46:06"),"user-since-copy":datetime("2010-04-24T05:46:06"),"friend-ids":{{7960737,27505427}},"employment":[{"organization-name":"Transhigh","start-date":date("2002-07-27"),"end-date":date("2004-07-28")}]}
+{"id":9740476,"id-copy":9740476,"alias":"Tucker","name":"TuckerRogers","user-since":datetime("2005-05-22T22:00:09"),"user-since-copy":datetime("2005-05-22T22:00:09"),"friend-ids":{{13095635,36113924,11767777,15169454,1692699,19622409,17110214}},"employment":[{"organization-name":"Zununoing","start-date":date("2009-03-24"),"end-date":date("2011-02-13")}]}
+{"id":11723506,"id-copy":11723506,"alias":"Odelia","name":"OdeliaPaul","user-since":datetime("2006-03-14T15:49:03"),"user-since-copy":datetime("2006-03-14T15:49:03"),"friend-ids":{{874326,37021972,27293893,40453006,44728117,338941,22832206,11391929,46420525}},"employment":[{"organization-name":"Zununoing","start-date":date("2012-05-05")}]}
+{"id":11627800,"id-copy":11627800,"alias":"Andrina","name":"AndrinaOrbell","user-since":datetime("2005-01-07T13:18:15"),"user-since-copy":datetime("2005-01-07T13:18:15"),"friend-ids":{{14378125}},"employment":[{"organization-name":"geomedia","start-date":date("2001-07-27"),"end-date":date("2009-01-26")}]}
+{"id":9842389,"id-copy":9842389,"alias":"Nicolas","name":"NicolasHynes","user-since":datetime("2005-08-10T23:35:18"),"user-since-copy":datetime("2005-08-10T23:35:18"),"friend-ids":{{40180500,33396487,26907885,4321366,10229201,41118923}},"employment":[{"organization-name":"Techitechi","start-date":date("2006-10-23"),"end-date":date("2010-03-11")}]}
+{"id":9733942,"id-copy":9733942,"alias":"Andra","name":"AndraConrad","user-since":datetime("2007-01-23T01:20:01"),"user-since-copy":datetime("2007-01-23T01:20:01"),"friend-ids":{{42791827,36987912,12650269,5310067,33419819,36880069,1146970,20314,10762565,20657888,31871678,42279496,9831201,4223369,46820320,21703772,1326858,21739453,20082273,12950360}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2002-03-21")}]}
+{"id":11476339,"id-copy":11476339,"alias":"Hopkin","name":"HopkinNicholas","user-since":datetime("2008-09-23T20:48:07"),"user-since-copy":datetime("2008-09-23T20:48:07"),"friend-ids":{{30021024,29046949,8412580,10700657,15739611,36768609}},"employment":[{"organization-name":"whitestreet","start-date":date("2004-01-02")}]}
+{"id":11582299,"id-copy":11582299,"alias":"Seward","name":"SewardReddish","user-since":datetime("2007-11-07T11:10:00"),"user-since-copy":datetime("2007-11-07T11:10:00"),"friend-ids":{{14793773,24447668,30727802,4757816,26139324,4433524,15974482}},"employment":[{"organization-name":"Fixdintex","start-date":date("2012-02-10")}]}
+{"id":11981266,"id-copy":11981266,"alias":"Meghann","name":"MeghannBatten","user-since":datetime("2008-06-04T14:25:11"),"user-since-copy":datetime("2008-06-04T14:25:11"),"friend-ids":{{39206334,28999157,22813777}},"employment":[{"organization-name":"Ganjastrip","start-date":date("2012-06-26")}]}
+{"id":11818252,"id-copy":11818252,"alias":"Sandee","name":"SandeeBlair","user-since":datetime("2008-12-22T20:09:56"),"user-since-copy":datetime("2008-12-22T20:09:56"),"friend-ids":{{35579096,13690328,19410347,10601941,13140634,19728850}},"employment":[{"organization-name":"kin-ron","start-date":date("2007-09-24")}]}
+{"id":11307037,"id-copy":11307037,"alias":"Brett","name":"BrettLeichter","user-since":datetime("2011-02-24T01:38:23"),"user-since-copy":datetime("2011-02-24T01:38:23"),"friend-ids":{{16273758,36959770,26721660}},"employment":[{"organization-name":"Tanzumbam","start-date":date("2011-11-23")}]}
+{"id":11357614,"id-copy":11357614,"alias":"Denys","name":"DenysMcintosh","user-since":datetime("2006-01-15T22:32:48"),"user-since-copy":datetime("2006-01-15T22:32:48"),"friend-ids":{{10713170,21699820,14949046,7935772,21404351,21078565,15867691,41676271,2655928,22987809,16585582,8318693,46886662,15081903,47617713,6317213,32997127}},"employment":[{"organization-name":"subtam","start-date":date("2008-08-28")}]}
+{"id":10607341,"id-copy":10607341,"alias":"Evander","name":"EvanderPycroft","user-since":datetime("2005-08-09T23:36:46"),"user-since-copy":datetime("2005-08-09T23:36:46"),"friend-ids":{{46200658,38004155}},"employment":[{"organization-name":"zoomplus","start-date":date("2004-06-13")}]}
+{"id":11214976,"id-copy":11214976,"alias":"Maxwell","name":"MaxwellBailey","user-since":datetime("2005-11-25T15:01:26"),"user-since-copy":datetime("2005-11-25T15:01:26"),"friend-ids":{{22027101,5782023,46909646,27593651,31079804,31989634,7337526,34757530,32792041}},"employment":[{"organization-name":"Vivaace","start-date":date("2011-04-04")}]}
+{"id":9160906,"id-copy":9160906,"alias":"Cathryn","name":"CathrynReamer","user-since":datetime("2010-10-08T06:24:51"),"user-since-copy":datetime("2010-10-08T06:24:51"),"friend-ids":{{30962953}},"employment":[{"organization-name":"zoomplus","start-date":date("2006-10-28"),"end-date":date("2010-03-14")}]}
+{"id":10464121,"id-copy":10464121,"alias":"Enriqueta","name":"EnriquetaHincken","user-since":datetime("2005-11-19T09:43:20"),"user-since-copy":datetime("2005-11-19T09:43:20"),"friend-ids":{{31238269,29421316,14426443,30128291,9926275,33523504,19113054,402505,12662005,36090974,8733776,18706660,14174144,46009221,17906304,41780430,21807110,22521282,21492740,34033053,16784027,11948555}},"employment":[{"organization-name":"zoomplus","start-date":date("2008-09-23")}]}
+{"id":11452525,"id-copy":11452525,"alias":"Suzanna","name":"SuzannaOlphert","user-since":datetime("2005-10-22T04:41:20"),"user-since-copy":datetime("2005-10-22T04:41:20"),"friend-ids":{{44250347,21517625,10831891,23365285,2000581,43387385,40167252,25288275,6768341,36116792,10670805}},"employment":[{"organization-name":"Villa-dox","start-date":date("2001-10-21"),"end-date":date("2005-03-11")}]}
+{"id":9403096,"id-copy":9403096,"alias":"Clarita","name":"ClaritaRitter","user-since":datetime("2007-11-18T14:11:04"),"user-since-copy":datetime("2007-11-18T14:11:04"),"friend-ids":{{11967380,17558867}},"employment":[{"organization-name":"zoomplus","start-date":date("2011-01-28"),"end-date":date("2011-05-05")}]}
+{"id":10267057,"id-copy":10267057,"alias":"Thomas","name":"ThomasCook","user-since":datetime("2008-03-02T23:04:31"),"user-since-copy":datetime("2008-03-02T23:04:31"),"friend-ids":{{23744020,25995598,40459051,27658275,10133202,11434833,29790727,1672639,19652058,18554997,37878642,48016133,46599310,37105777,36004129,6402365,9889815,29589019,1497208,19269802,43383394,30936085}},"employment":[{"organization-name":"Statcode","start-date":date("2008-05-14"),"end-date":date("2008-07-10")}]}
+{"id":10547020,"id-copy":10547020,"alias":"Reita","name":"ReitaBlunt","user-since":datetime("2006-01-18T16:51:49"),"user-since-copy":datetime("2006-01-18T16:51:49"),"friend-ids":{{34373903,36464697,37171525,19138424,24675436,16269152,43940985,2735762,32760257,42561749,45516984,39110107,21610913,1805884,3342035,40703512,11665984,29345992,41497492,30054924,18098215}},"employment":[{"organization-name":"Xx-drill","start-date":date("2001-12-01")}]}
+{"id":10287028,"id-copy":10287028,"alias":"Wilfred","name":"WilfredChurchill","user-since":datetime("2007-08-01T14:14:25"),"user-since-copy":datetime("2007-08-01T14:14:25"),"friend-ids":{{38355737,39891840,41036196,39165706,1155288,15280633,9744287,11567914,11225763,2297894,14386027,67174,28097703,28721858,6504409,6743503,22860419,17773814,34697084,5419586,45771084}},"employment":[{"organization-name":"Villa-tech","start-date":date("2002-08-08")}]}
+{"id":11145823,"id-copy":11145823,"alias":"Rebeccah","name":"RebeccahTodd","user-since":datetime("2007-03-25T15:13:08"),"user-since-copy":datetime("2007-03-25T15:13:08"),"friend-ids":{{46132741,11527757,27573172,45663865,45572803,30569464,31892238}},"employment":[{"organization-name":"whitestreet","start-date":date("2012-04-07")}]}
+{"id":10640851,"id-copy":10640851,"alias":"Tabitha","name":"TabithaWhitten","user-since":datetime("2010-01-28T14:25:58"),"user-since-copy":datetime("2010-01-28T14:25:58"),"friend-ids":{{42792549,5330514,24582133,43384590,38083439,31221232,18064537,21736064,7919520,18998284,20165148,28492287,21987533,23638155}},"employment":[{"organization-name":"sonstreet","start-date":date("2006-06-18"),"end-date":date("2007-07-20")}]}
+{"id":10298530,"id-copy":10298530,"alias":"Natalee","name":"NataleeBell","user-since":datetime("2010-09-07T14:14:59"),"user-since-copy":datetime("2010-09-07T14:14:59"),"friend-ids":{{36077399,47946678,4189158,42122618,14179077,26433248,25903252,23116624,33542934,1071320,31914369,28408518,40811454,19212473,25057330,42758915}},"employment":[{"organization-name":"Sancone","start-date":date("2012-02-17")}]}
+{"id":10931563,"id-copy":10931563,"alias":"Laraine","name":"LaraineCountryman","user-since":datetime("2012-03-17T17:06:59"),"user-since-copy":datetime("2012-03-17T17:06:59"),"friend-ids":{{17266368,75990,37678426,43207424,37434492,26338447,33450799,5401110,44962643,5514847}},"employment":[{"organization-name":"Dandamace","start-date":date("2008-09-08")}]}
+{"id":11244283,"id-copy":11244283,"alias":"Erica","name":"EricaTilton","user-since":datetime("2005-12-10T16:37:41"),"user-since-copy":datetime("2005-12-10T16:37:41"),"friend-ids":{{9476551,22631836,44127713,32391437,19413944,4263930,17603111,24077268,31120069,30869992,6040985,3918705,17640663,22515182}},"employment":[{"organization-name":"Striptaxon","start-date":date("2002-02-05"),"end-date":date("2003-07-03")}]}
+{"id":9669178,"id-copy":9669178,"alias":"Gerard","name":"GerardBeck","user-since":datetime("2011-04-24T15:49:24"),"user-since-copy":datetime("2011-04-24T15:49:24"),"friend-ids":{{30087138,44736614,1531569}},"employment":[{"organization-name":"Fix-touch","start-date":date("2003-09-25"),"end-date":date("2005-06-28")}]}
+{"id":11920078,"id-copy":11920078,"alias":"Alane","name":"AlaneRichter","user-since":datetime("2005-04-12T04:06:03"),"user-since-copy":datetime("2005-04-12T04:06:03"),"friend-ids":{{18326190,34366549,13047472,29553920,6210406,41865352,26108964,15042193,33225025,7014329,11051157,37032436,8025322,21902099,22953955,42645725,29144585}},"employment":[{"organization-name":"Streettax","start-date":date("2006-04-24")}]}
+{"id":11989228,"id-copy":11989228,"alias":"Jaden","name":"JadenKelley","user-since":datetime("2006-11-12T15:45:55"),"user-since-copy":datetime("2006-11-12T15:45:55"),"friend-ids":{{39881086,47143027,9394301,17338199,16961896,6602092,46708527,24050942,20543677,13309656}},"employment":[{"organization-name":"sonstreet","start-date":date("2003-10-19")}]}
+{"id":10478512,"id-copy":10478512,"alias":"Remona","name":"RemonaPittman","user-since":datetime("2007-06-19T12:20:07"),"user-since-copy":datetime("2007-06-19T12:20:07"),"friend-ids":{{12750727}},"employment":[{"organization-name":"Fax-fax","start-date":date("2001-02-02")}]}
+{"id":11990740,"id-copy":11990740,"alias":"Vernon","name":"VernonBarnes","user-since":datetime("2005-05-25T09:07:06"),"user-since-copy":datetime("2005-05-25T09:07:06"),"friend-ids":{{44677447,20354746,30157224,29686873,9413456,11656099,25404439,24706566,45005726,22096097,29868918,12109246,38948331,2643312,41565707,17566751,8045341,25358960,43614095,28262168,14405467,22519550}},"employment":[{"organization-name":"goldendexon","start-date":date("2012-01-23")}]}
+{"id":9210847,"id-copy":9210847,"alias":"Kristeen","name":"KristeenShaffer","user-since":datetime("2008-01-04T12:31:50"),"user-since-copy":datetime("2008-01-04T12:31:50"),"friend-ids":{{662954,18313322,10737685,5498351,24795605,4497605,45729062,31007969,16211490,19408104,5882137,12084923,14143383,31263672,32404691,8973685,32756191,3822704}},"employment":[{"organization-name":"Rungozoom","start-date":date("2009-12-07"),"end-date":date("2010-02-08")}]}
+{"id":11187373,"id-copy":11187373,"alias":"Garfield","name":"GarfieldWible","user-since":datetime("2009-06-19T05:22:16"),"user-since-copy":datetime("2009-06-19T05:22:16"),"friend-ids":{{24453777,20841948,12224610,30351943,17826670,36119836,27850423,4004658,42610631,25893845,46022891,33018964,37844844,1705377,38811008,36802000}},"employment":[{"organization-name":"Plexlane","start-date":date("2000-02-22")}]}
+{"id":10026061,"id-copy":10026061,"alias":"Nonie","name":"NonieChappel","user-since":datetime("2007-06-22T10:06:38"),"user-since-copy":datetime("2007-06-22T10:06:38"),"friend-ids":{{38760716,16809503,6592849,3736630,32388289,40487693,27146403,22621793,35615399,10839746,693037,25222841,46448329,40740448,21652202,30069817,21957966}},"employment":[{"organization-name":"Freshfix","start-date":date("2010-08-19"),"end-date":date("2010-08-17")}]}
+{"id":9638248,"id-copy":9638248,"alias":"Azucena","name":"AzucenaEmrick","user-since":datetime("2005-12-04T00:15:40"),"user-since-copy":datetime("2005-12-04T00:15:40"),"friend-ids":{{37210744,43097413,2901403,24492031,7887583,42518446,28555003,20402754,5506767,22982986,21168589,37638670,30930177,43662522,45627167,13450586,36757137,46663990}},"employment":[{"organization-name":"Sublamdox","start-date":date("2002-07-07"),"end-date":date("2006-06-11")}]}
+{"id":10832305,"id-copy":10832305,"alias":"Briony","name":"BrionyBaldwin","user-since":datetime("2011-03-03T22:00:38"),"user-since-copy":datetime("2011-03-03T22:00:38"),"friend-ids":{{20436897,36519715,35325917,31686319,2644929,3401668,39344422,18601722,40274111,30032679,9312830,5581755,41164101,35883066,8274432,4315219,26200418,43810182,44718149,6387153,43086214,39558538,36036905,25715671}},"employment":[{"organization-name":"Indiex","start-date":date("2007-01-21"),"end-date":date("2008-02-25")}]}
+{"id":9139057,"id-copy":9139057,"alias":"Esther","name":"EstherUllman","user-since":datetime("2010-01-05T19:25:44"),"user-since-copy":datetime("2010-01-05T19:25:44"),"friend-ids":{{25401186,25915246,33727208,17431690,24541706,19998503,42399029,30405906,20023918,9788811,32513474,14919034,10073867,9309154,1423378,37386209,16346279,45167618,34716280,29023237,20639001,332097,28344544}},"employment":[{"organization-name":"Icerunin","start-date":date("2003-09-05"),"end-date":date("2009-10-17")}]}
+{"id":10150873,"id-copy":10150873,"alias":"Shanice","name":"ShaniceReiss","user-since":datetime("2005-07-07T09:46:00"),"user-since-copy":datetime("2005-07-07T09:46:00"),"friend-ids":{{29208488,6994033,13074568,31547206,2547580,15915539,37448883,38739687,33246865,28231547,33861348,44929557,13977747,44297013,22367804}},"employment":[{"organization-name":"Plexlane","start-date":date("2002-09-07"),"end-date":date("2006-04-23")}]}
+{"id":11458594,"id-copy":11458594,"alias":"Rosaline","name":"RosalineHawker","user-since":datetime("2006-06-07T01:36:07"),"user-since-copy":datetime("2006-06-07T01:36:07"),"friend-ids":{{13674953,43755185,20151836,40023637,35564429,45196295,33392303,2080473,6786170,42815553,10811200,5050190,20987923,32613675}},"employment":[{"organization-name":"Ganjatax","start-date":date("2002-06-05")}]}
+{"id":11801005,"id-copy":11801005,"alias":"Jacques","name":"JacquesWhitling","user-since":datetime("2007-05-20T05:42:21"),"user-since-copy":datetime("2007-05-20T05:42:21"),"friend-ids":{{45134681,48016178}},"employment":[{"organization-name":"Basecone","start-date":date("2006-12-07")}]}
+{"id":9812020,"id-copy":9812020,"alias":"Elias","name":"EliasBuck","user-since":datetime("2012-08-03T07:52:34"),"user-since-copy":datetime("2012-08-03T07:52:34"),"friend-ids":{{}},"employment":[{"organization-name":"Doncare","start-date":date("2008-07-05"),"end-date":date("2008-12-18")}]}
+{"id":10404706,"id-copy":10404706,"alias":"Rylan","name":"RylanEmrick","user-since":datetime("2008-11-23T00:55:36"),"user-since-copy":datetime("2008-11-23T00:55:36"),"friend-ids":{{17936230,20908773,34834317,26134774,3534090,7699389,11743997,37809096,23228338,19069026,662582,40839640,26706968,42711557,28658968,39161015,29201879,7516443,21802464,16456657,32689464}},"employment":[{"organization-name":"over-it","start-date":date("2012-02-16")}]}
+{"id":11412382,"id-copy":11412382,"alias":"Gosse","name":"GosseSutton","user-since":datetime("2011-01-07T02:19:16"),"user-since-copy":datetime("2011-01-07T02:19:16"),"friend-ids":{{25790586,42348812,39275252,32764855,11642271,15982736,21971689,13168697,38246675,40514837,20840965}},"employment":[{"organization-name":"Salthex","start-date":date("2010-12-18"),"end-date":date("2011-01-09")}]}
+{"id":11588467,"id-copy":11588467,"alias":"Soon","name":"SoonHays","user-since":datetime("2011-12-21T05:33:54"),"user-since-copy":datetime("2011-12-21T05:33:54"),"friend-ids":{{659930}},"employment":[{"organization-name":"Plexlane","start-date":date("2009-05-20"),"end-date":date("2009-07-16")}]}
+{"id":11015908,"id-copy":11015908,"alias":"Giuseppe","name":"GiuseppeWard","user-since":datetime("2008-09-14T16:37:40"),"user-since-copy":datetime("2008-09-14T16:37:40"),"friend-ids":{{9972151,40271551,46207899,29987388,19876511,47546614,17051350,1579198,2151480,26507940,18177808,25866392,40253780}},"employment":[{"organization-name":"Scotcity","start-date":date("2008-02-24")}]}
+{"id":9480964,"id-copy":9480964,"alias":"Ava","name":"AvaCross","user-since":datetime("2005-11-03T14:59:13"),"user-since-copy":datetime("2005-11-03T14:59:13"),"friend-ids":{{9693959,3138151,20631444,8672727,33701530,14630539,38539482,3066915,30934733,38630163,25673376}},"employment":[{"organization-name":"Codetechno","start-date":date("2000-06-05"),"end-date":date("2000-10-06")}]}
+{"id":11932807,"id-copy":11932807,"alias":"Sheridan","name":"SheridanCarr","user-since":datetime("2009-05-17T01:39:53"),"user-since-copy":datetime("2009-05-17T01:39:53"),"friend-ids":{{12836351,10066178,40881248,3744364,18904729,10238846,27947251,23407801,39613208,34468026,20801656,46114253,26807188,13084266,27104805,27016320,25825154,16782132,29528918}},"employment":[{"organization-name":"U-ron","start-date":date("2001-09-04"),"end-date":date("2005-01-15")}]}
+{"id":10514095,"id-copy":10514095,"alias":"Chantelle","name":"ChantelleCatleay","user-since":datetime("2008-10-23T00:05:15"),"user-since-copy":datetime("2008-10-23T00:05:15"),"friend-ids":{{11871759,1505524,45483061,31479407,15112731,41816114,22650998}},"employment":[{"organization-name":"Hatcom","start-date":date("2008-06-14")}]}
+{"id":9880696,"id-copy":9880696,"alias":"Cynthia","name":"CynthiaSeidner","user-since":datetime("2006-03-17T01:36:33"),"user-since-copy":datetime("2006-03-17T01:36:33"),"friend-ids":{{47318799,28282167}},"employment":[{"organization-name":"Newfase","start-date":date("2008-07-02"),"end-date":date("2010-11-25")}]}
+{"id":10391077,"id-copy":10391077,"alias":"Tracy","name":"TracyHiles","user-since":datetime("2005-11-19T21:08:51"),"user-since-copy":datetime("2005-11-19T21:08:51"),"friend-ids":{{27119048,1983772,38766385,35631268,14736954,7586158,45840742,27211063,33946244,1590669,22363833,19668917,12778790,31993728,4498870,68121,13591025,13285639}},"employment":[{"organization-name":"strongex","start-date":date("2012-07-12")}]}
+{"id":9481756,"id-copy":9481756,"alias":"Esmaralda","name":"EsmaraldaAgg","user-since":datetime("2012-06-26T19:57:38"),"user-since-copy":datetime("2012-06-26T19:57:38"),"friend-ids":{{40976868}},"employment":[{"organization-name":"zoomplus","start-date":date("2008-11-26"),"end-date":date("2008-01-13")}]}
+{"id":10151953,"id-copy":10151953,"alias":"Howard","name":"HowardHoopengarner","user-since":datetime("2006-07-23T01:43:57"),"user-since-copy":datetime("2006-07-23T01:43:57"),"friend-ids":{{32564548,19333543,27610653,27936980,7471201,1353451,30864511,41582907,22918030,6011307,21622284,44695813,34728110,33062051,29420834,37472592,3655974,34618485,21615748,14107596,15317302,21805666,4563480}},"employment":[{"organization-name":"Doncare","start-date":date("2012-06-08")}]}
+{"id":10277731,"id-copy":10277731,"alias":"Gallagher","name":"GallagherMagor","user-since":datetime("2007-07-02T07:37:02"),"user-since-copy":datetime("2007-07-02T07:37:02"),"friend-ids":{{22730683,9352614,42748868,24014877,21749502,30751403,41768964,13317192,31877814,35318552,26843471,21232937,11268529,21902785}},"employment":[{"organization-name":"Canline","start-date":date("2006-09-03")}]}
+{"id":11646016,"id-copy":11646016,"alias":"Millard","name":"MillardCribbs","user-since":datetime("2012-07-01T13:28:56"),"user-since-copy":datetime("2012-07-01T13:28:56"),"friend-ids":{{29358027,24800104,1146956,29116484,12223225,6324161,46576675}},"employment":[{"organization-name":"Inchdox","start-date":date("2004-04-28")}]}
+{"id":9012778,"id-copy":9012778,"alias":"Godfrey","name":"GodfreyBraun","user-since":datetime("2010-03-18T19:15:53"),"user-since-copy":datetime("2010-03-18T19:15:53"),"friend-ids":{{3867712,22078166}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2005-10-02")}]}
+{"id":9074290,"id-copy":9074290,"alias":"Riley","name":"RileyBode","user-since":datetime("2010-11-20T01:12:36"),"user-since-copy":datetime("2010-11-20T01:12:36"),"friend-ids":{{}},"employment":[{"organization-name":"Tripplelane","start-date":date("2004-10-12")}]}
+{"id":9223375,"id-copy":9223375,"alias":"Anne","name":"AnneMoore","user-since":datetime("2010-07-16T22:06:20"),"user-since-copy":datetime("2010-07-16T22:06:20"),"friend-ids":{{45553359,40589681,9461257,39253068,14447226,37656564,37047377,34855985}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2011-04-25")}]}
+{"id":9897094,"id-copy":9897094,"alias":"Raynard","name":"RaynardWade","user-since":datetime("2010-05-12T19:44:55"),"user-since-copy":datetime("2010-05-12T19:44:55"),"friend-ids":{{21246472,34504200,43744110,30518742,1016046,17644601,47173648,11643135,22382871,38535297,17156487,30328939,14770807,9365820,36893585,30122942,37610936,44304872}},"employment":[{"organization-name":"kin-ron","start-date":date("2004-02-18")}]}
+{"id":10189600,"id-copy":10189600,"alias":"Melisa","name":"MelisaGarry","user-since":datetime("2010-05-10T10:35:49"),"user-since-copy":datetime("2010-05-10T10:35:49"),"friend-ids":{{18172527,26205741,32077713,41214698,33783052,5734397,46101468,30210046,27425699}},"employment":[{"organization-name":"Tripplelane","start-date":date("2011-05-20"),"end-date":date("2011-07-20")}]}
+{"id":10001047,"id-copy":10001047,"alias":"Rodger","name":"RodgerRifler","user-since":datetime("2009-12-08T18:34:21"),"user-since-copy":datetime("2009-12-08T18:34:21"),"friend-ids":{{41832587,41015556,17486735,38428485,29774516,38574837,2061546,46972940,25654449,776023,1164809,34242171,9752352,1088591,26406961,7270316,36371574,24413303,36287374,43343719,6830709,2919772,41313339}},"employment":[{"organization-name":"Fixdintex","start-date":date("2003-02-08")}]}
+{"id":10573795,"id-copy":10573795,"alias":"Neil","name":"NeilMilne","user-since":datetime("2005-11-15T02:57:46"),"user-since-copy":datetime("2005-11-15T02:57:46"),"friend-ids":{{33469327,4261514,43412669,17289131,27535421,15267017,14005060}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2001-11-13"),"end-date":date("2001-10-28")}]}
+{"id":10561624,"id-copy":10561624,"alias":"Marielle","name":"MarielleBrandenburg","user-since":datetime("2005-07-17T10:28:02"),"user-since-copy":datetime("2005-07-17T10:28:02"),"friend-ids":{{1231477,14598987}},"employment":[{"organization-name":"tresline","start-date":date("2000-03-06"),"end-date":date("2005-09-25")}]}
+{"id":11781745,"id-copy":11781745,"alias":"Merv","name":"MervStocker","user-since":datetime("2008-10-15T03:41:54"),"user-since-copy":datetime("2008-10-15T03:41:54"),"friend-ids":{{26394519,2599602,40237077,43817129,30392481,43051494,36128635,35974184,37237292,7775912,11569464,9112021,26837692,11548106,29331601,11126182,18076463,33866145,22408972,42318835,47199541,26807788}},"employment":[{"organization-name":"Xx-drill","start-date":date("2005-01-15"),"end-date":date("2008-02-18")}]}
+{"id":10905802,"id-copy":10905802,"alias":"Jamika","name":"JamikaJowers","user-since":datetime("2007-05-24T01:31:04"),"user-since-copy":datetime("2007-05-24T01:31:04"),"friend-ids":{{16476991,9041491,10867973,18057276,13716912,184635,47717267,37995364}},"employment":[{"organization-name":"Quadlane","start-date":date("2008-08-20")}]}
+{"id":10824484,"id-copy":10824484,"alias":"Linda","name":"LindaStanfield","user-since":datetime("2009-03-03T12:54:55"),"user-since-copy":datetime("2009-03-03T12:54:55"),"friend-ids":{{39164563,20321780,19901289,37969494,15051354,42576590,14550253,33649901,6008727,17749643,7792769,18652053,8565400,43899372,7433016,42506713}},"employment":[{"organization-name":"Techitechi","start-date":date("2012-03-21")}]}
+{"id":9372871,"id-copy":9372871,"alias":"Emerson","name":"EmersonSell","user-since":datetime("2010-01-25T11:12:56"),"user-since-copy":datetime("2010-01-25T11:12:56"),"friend-ids":{{13800934,24493814}},"employment":[{"organization-name":"Keytech","start-date":date("2004-02-14"),"end-date":date("2005-11-07")}]}
+{"id":10250857,"id-copy":10250857,"alias":"Kandi","name":"KandiFranks","user-since":datetime("2010-11-24T19:47:41"),"user-since-copy":datetime("2010-11-24T19:47:41"),"friend-ids":{{44991748,27655130,7925482,33419150,18275478}},"employment":[{"organization-name":"Kongreen","start-date":date("2002-04-08")}]}
+{"id":11591713,"id-copy":11591713,"alias":"Nannie","name":"NannieDiller","user-since":datetime("2008-11-27T08:31:02"),"user-since-copy":datetime("2008-11-27T08:31:02"),"friend-ids":{{26059738,32515289,13702345,16949001,10188160,30251286}},"employment":[{"organization-name":"Zuncan","start-date":date("2000-11-27")}]}
+{"id":11366131,"id-copy":11366131,"alias":"Cayley","name":"CayleyGronko","user-since":datetime("2005-03-06T13:24:19"),"user-since-copy":datetime("2005-03-06T13:24:19"),"friend-ids":{{26623267,47792710,27975124,19721566,45092752,32954140,25835098}},"employment":[{"organization-name":"Alphadax","start-date":date("2006-09-14"),"end-date":date("2010-06-02")}]}
+{"id":10809322,"id-copy":10809322,"alias":"Alden","name":"AldenHiggens","user-since":datetime("2011-02-06T01:31:58"),"user-since-copy":datetime("2011-02-06T01:31:58"),"friend-ids":{{44750450,24564153,42513064,33316253,21036452,27132567,29231674,18040424,36564417,17474605,14126628,18988855,35594147,35685289,40967850}},"employment":[{"organization-name":"zoomplus","start-date":date("2002-09-26")}]}
+{"id":11571085,"id-copy":11571085,"alias":"Reina","name":"ReinaWheeler","user-since":datetime("2010-04-28T08:05:29"),"user-since-copy":datetime("2010-04-28T08:05:29"),"friend-ids":{{25357083,40592075,10585644,33173927,42515085}},"employment":[{"organization-name":"Zununoing","start-date":date("2000-08-03")}]}
+{"id":10797166,"id-copy":10797166,"alias":"Alethea","name":"AletheaMills","user-since":datetime("2011-01-10T03:06:16"),"user-since-copy":datetime("2011-01-10T03:06:16"),"friend-ids":{{25077851,2396037,25762626,31358162,41492027,31211140,38478662,9688210,16865534,4209161,19863828,23760993,36041139,46184667}},"employment":[{"organization-name":"Medflex","start-date":date("2004-05-04")}]}
+{"id":9168649,"id-copy":9168649,"alias":"Harmony","name":"HarmonyMackendoerfer","user-since":datetime("2006-06-25T21:01:50"),"user-since-copy":datetime("2006-06-25T21:01:50"),"friend-ids":{{197057,11973988,2042364,21282964,25761405,10180346,39780287,39243722,2984620,7756400,21311572,21013939,16998045,39135533,47720897,20316953}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2005-12-17"),"end-date":date("2009-07-11")}]}
+{"id":9407710,"id-copy":9407710,"alias":"Todd","name":"ToddStall","user-since":datetime("2009-09-21T02:18:16"),"user-since-copy":datetime("2009-09-21T02:18:16"),"friend-ids":{{46998635,14217621,8062100,47498395,37234901,41039045,37635206,42173831,24149948}},"employment":[{"organization-name":"Inchex","start-date":date("2009-09-27"),"end-date":date("2009-07-21")}]}
+{"id":10772929,"id-copy":10772929,"alias":"Hugh","name":"HughTrout","user-since":datetime("2008-01-24T03:16:55"),"user-since-copy":datetime("2008-01-24T03:16:55"),"friend-ids":{{39704817,19656412,37084896,5219803,23455492,14248249,26973609,4607440,25844255,3032226,45432192,47011338,41460367,28779211,31780563,31808543,29732190,1264228,7989711,38397890,7638694,3002993,8960147,46258407}},"employment":[{"organization-name":"Vivaace","start-date":date("2010-08-02"),"end-date":date("2010-05-08")}]}
+{"id":11233525,"id-copy":11233525,"alias":"Syd","name":"SydSauter","user-since":datetime("2010-12-18T02:44:55"),"user-since-copy":datetime("2010-12-18T02:44:55"),"friend-ids":{{6312313,17431246,36729581,3715101,39534341,10333995,36042764,14014852,27375328,17089631,24066240,42616402,34049424,29807262,25669160,43435752,46702290,27418631,13587383,14811241}},"employment":[{"organization-name":"Transhigh","start-date":date("2010-06-21")}]}
+{"id":11741821,"id-copy":11741821,"alias":"Cal","name":"CalHowe","user-since":datetime("2005-12-27T20:26:31"),"user-since-copy":datetime("2005-12-27T20:26:31"),"friend-ids":{{45052138}},"employment":[{"organization-name":"Newphase","start-date":date("2006-12-10"),"end-date":date("2006-02-25")}]}
+{"id":9761152,"id-copy":9761152,"alias":"Royle","name":"RoyleStewart","user-since":datetime("2010-05-15T17:14:18"),"user-since-copy":datetime("2010-05-15T17:14:18"),"friend-ids":{{21868661,15545005,11285872,45768523,12486235}},"employment":[{"organization-name":"Technohow","start-date":date("2002-11-20")}]}
+{"id":11694928,"id-copy":11694928,"alias":"Anne","name":"AnnePritchard","user-since":datetime("2005-05-25T23:02:45"),"user-since-copy":datetime("2005-05-25T23:02:45"),"friend-ids":{{4000537,32410978,2682612,1214946,38250943,36272447,14182545,27782322,2714608,38315875}},"employment":[{"organization-name":"U-ron","start-date":date("2011-02-22"),"end-date":date("2011-11-07")}]}
+{"id":11049274,"id-copy":11049274,"alias":"Fitz","name":"FitzBeail","user-since":datetime("2012-08-10T03:25:57"),"user-since-copy":datetime("2012-08-10T03:25:57"),"friend-ids":{{39403330,13441324,723509,34025727,23266816,33898717,11053310,14582395,38435153,45855468,45712821}},"employment":[{"organization-name":"U-ron","start-date":date("2004-02-16"),"end-date":date("2007-01-07")}]}
+{"id":11695309,"id-copy":11695309,"alias":"Petula","name":"PetulaTanner","user-since":datetime("2011-12-23T13:29:44"),"user-since-copy":datetime("2011-12-23T13:29:44"),"friend-ids":{{39411346,33118908,44553603}},"employment":[{"organization-name":"Techitechi","start-date":date("2003-02-26"),"end-date":date("2007-11-12")}]}
+{"id":9050866,"id-copy":9050866,"alias":"Jimmie","name":"JimmieBicknell","user-since":datetime("2007-02-15T16:39:19"),"user-since-copy":datetime("2007-02-15T16:39:19"),"friend-ids":{{17248854,13830961,10571254,637235,18219702,4541511,42876025,19679892,14009802,15312402,20914286,41969971,39807443,5990836,1594551,25853135,25021671,21604624,47574478}},"employment":[{"organization-name":"Ontohothex","start-date":date("2001-04-09")}]}
+{"id":9818617,"id-copy":9818617,"alias":"Elwyn","name":"ElwynEndsley","user-since":datetime("2012-04-12T18:14:54"),"user-since-copy":datetime("2012-04-12T18:14:54"),"friend-ids":{{44007613,15744997,9366576,44776374,19082361,9967101,25247773,20407697}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2003-09-09")}]}
+{"id":10861183,"id-copy":10861183,"alias":"Zilla","name":"ZillaOneal","user-since":datetime("2008-03-12T23:37:18"),"user-since-copy":datetime("2008-03-12T23:37:18"),"friend-ids":{{26262188,17172669,43068853,47767064,34552281,33602720,35448839,6347557,11913432,45186875,10451537,46881437,27965706}},"employment":[{"organization-name":"Tripplelane","start-date":date("2008-09-03"),"end-date":date("2009-07-22")}]}
+{"id":11573350,"id-copy":11573350,"alias":"Sommer","name":"SommerGregory","user-since":datetime("2007-08-25T21:50:51"),"user-since-copy":datetime("2007-08-25T21:50:51"),"friend-ids":{{6622046,40071999,24631984,42427860,13378139,27659078,32813734,20145238,15342806,9562288,24211264,29951003,3620479,43701781,22474191,6298296,4047189,27133942,8058121,9928231,31835361,6234235,6100660,1575061}},"employment":[{"organization-name":"Kongreen","start-date":date("2010-12-09"),"end-date":date("2010-01-16")}]}
+{"id":10297336,"id-copy":10297336,"alias":"Gayelord","name":"GayelordCypret","user-since":datetime("2005-09-28T10:01:31"),"user-since-copy":datetime("2005-09-28T10:01:31"),"friend-ids":{{43657472,21189656,43018991,42333420,27203617,12389046,44062328,15441240,31806533,44999377,30592890,12304605,6752099,9488471,5719065,16290550,23175098,6432261}},"employment":[{"organization-name":"Dandamace","start-date":date("2001-05-15")}]}
+{"id":10985830,"id-copy":10985830,"alias":"Spencer","name":"SpencerWilo","user-since":datetime("2010-03-02T07:41:59"),"user-since-copy":datetime("2010-03-02T07:41:59"),"friend-ids":{{5766878,20551454,27297902,44757901,7660518,28072828,6387548,6276027,40692560,36168648,24514885,40791549,15536640,23757967,19875372}},"employment":[{"organization-name":"Inchex","start-date":date("2004-04-14"),"end-date":date("2009-02-17")}]}
+{"id":9081124,"id-copy":9081124,"alias":"Aureole","name":"AureoleChappel","user-since":datetime("2005-03-24T18:14:35"),"user-since-copy":datetime("2005-03-24T18:14:35"),"friend-ids":{{16199402,2970920}},"employment":[{"organization-name":"Transhigh","start-date":date("2005-07-19"),"end-date":date("2011-04-02")}]}
+{"id":11780581,"id-copy":11780581,"alias":"Simona","name":"SimonaDrumm","user-since":datetime("2010-09-10T00:03:56"),"user-since-copy":datetime("2010-09-10T00:03:56"),"friend-ids":{{14930223,14107902,18276584,12824637,44738306,252529,17504815,26802467,33312123,15516170,9060069,42300993,15746839,61844,1966381,31284798,40145954,31282156,15764470,9894586,41833755}},"employment":[{"organization-name":"Goldcity","start-date":date("2011-03-27")}]}
+{"id":10563310,"id-copy":10563310,"alias":"Justina","name":"JustinaHall","user-since":datetime("2010-08-24T08:57:45"),"user-since-copy":datetime("2010-08-24T08:57:45"),"friend-ids":{{42796179,25994871,35439919,28722419,7189994,41505357,35095639,14693797,36519323,32598167,6323551,14565174,35997662,9705559,3996730}},"employment":[{"organization-name":"Canline","start-date":date("2011-02-20"),"end-date":date("2011-05-05")}]}
+{"id":10320979,"id-copy":10320979,"alias":"Giuseppe","name":"GiuseppePorter","user-since":datetime("2006-10-21T21:56:23"),"user-since-copy":datetime("2006-10-21T21:56:23"),"friend-ids":{{34102109,41585396,8170669,7376463,11841426,6745396,35637670,38513040,26085708,7577827,4793535,31185038,9126,502656,18672743,27688404,19846788,47731814,42609593}},"employment":[{"organization-name":"Viatechi","start-date":date("2012-06-08")}]}
+{"id":11670331,"id-copy":11670331,"alias":"Deetta","name":"DeettaCrom","user-since":datetime("2008-04-01T00:12:47"),"user-since-copy":datetime("2008-04-01T00:12:47"),"friend-ids":{{34871046,45366633,40484162,45505621,47279131,5464046,18435436,24937987,18253019,5870229,46379232,13988659,37921800,2085103,21652843,4802881,11658526,40771399,32938488,8409007,27179341,4496744}},"employment":[{"organization-name":"Dancode","start-date":date("2003-10-21"),"end-date":date("2008-06-06")}]}
+{"id":11448565,"id-copy":11448565,"alias":"Martie","name":"MartiePoley","user-since":datetime("2010-07-02T14:37:46"),"user-since-copy":datetime("2010-07-02T14:37:46"),"friend-ids":{{45198632,14347405,14595348,4990646,44745176,21949325,9155582,3970455,10097690,35781298,46746615,35535590,16561713,31169880,22467369}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2006-02-08")}]}
+{"id":9267007,"id-copy":9267007,"alias":"Perla","name":"PerlaCox","user-since":datetime("2009-04-14T20:56:37"),"user-since-copy":datetime("2009-04-14T20:56:37"),"friend-ids":{{8937408,4640163,41404266,15668694,21004833,12635405,40379208,18641131,14014264,39008348,36559306,26261953,3593955,13559713,34525259}},"employment":[{"organization-name":"Ganjatax","start-date":date("2003-07-02")}]}
+{"id":10745974,"id-copy":10745974,"alias":"Gavin","name":"GavinWard","user-since":datetime("2008-11-23T02:59:13"),"user-since-copy":datetime("2008-11-23T02:59:13"),"friend-ids":{{45290227,46308273,4478698,27613190,34907694,36182643}},"employment":[{"organization-name":"Dandamace","start-date":date("2001-01-01"),"end-date":date("2011-01-17")}]}
+{"id":11113168,"id-copy":11113168,"alias":"Daphne","name":"DaphneHindman","user-since":datetime("2011-11-09T02:55:42"),"user-since-copy":datetime("2011-11-09T02:55:42"),"friend-ids":{{194785,11696942,23072861,37052204,17574763,14099428,44155581}},"employment":[{"organization-name":"Basecone","start-date":date("2002-07-16"),"end-date":date("2006-11-08")}]}
+{"id":10348309,"id-copy":10348309,"alias":"Bernard","name":"BernardAltman","user-since":datetime("2010-09-23T09:08:33"),"user-since-copy":datetime("2010-09-23T09:08:33"),"friend-ids":{{7859503,40438517,7050233,41735514,8274833,12496793,41853402,23751827,23485505,35520895,17406459,20238814,42333149}},"employment":[{"organization-name":"goldendexon","start-date":date("2007-07-27")}]}
+{"id":10552405,"id-copy":10552405,"alias":"Les","name":"LesBarth","user-since":datetime("2008-04-02T11:02:37"),"user-since-copy":datetime("2008-04-02T11:02:37"),"friend-ids":{{33645432,43039707}},"employment":[{"organization-name":"sonstreet","start-date":date("2000-09-18")}]}
+{"id":10065250,"id-copy":10065250,"alias":"Debbie","name":"DebbieBrinigh","user-since":datetime("2012-01-05T15:05:48"),"user-since-copy":datetime("2012-01-05T15:05:48"),"friend-ids":{{23794420,31166549,3372724,35955226,45241312,33488036,17353508,10094234,12751868}},"employment":[{"organization-name":"freshdox","start-date":date("2000-06-28"),"end-date":date("2005-06-03")}]}
+{"id":9767755,"id-copy":9767755,"alias":"Joel","name":"JoelHoopengarner","user-since":datetime("2012-01-19T13:22:46"),"user-since-copy":datetime("2012-01-19T13:22:46"),"friend-ids":{{41934568,20874721,33807743}},"employment":[{"organization-name":"physcane","start-date":date("2012-01-21"),"end-date":date("2012-06-09")}]}
+{"id":11355979,"id-copy":11355979,"alias":"Sal","name":"SalChapman","user-since":datetime("2012-07-23T17:03:04"),"user-since-copy":datetime("2012-07-23T17:03:04"),"friend-ids":{{4959799,33919735,33624568,9885012,16788595,39510500,34856818,22167281,44317359,45181449,43901851,42402339,9573000,16655168}},"employment":[{"organization-name":"Ransaofan","start-date":date("2006-12-10")}]}
+{"id":10613617,"id-copy":10613617,"alias":"Jeanie","name":"JeanieEiford","user-since":datetime("2007-02-09T12:16:09"),"user-since-copy":datetime("2007-02-09T12:16:09"),"friend-ids":{{24843944,3651507,25077638,18662161,46723847,31558857,11235682,15640606,31889112,45342233,25865191,1530020,39187188,4939030,19220487,19619126,25284665,1206869,40740763}},"employment":[{"organization-name":"Sublamdox","start-date":date("2001-07-22")}]}
+{"id":11872177,"id-copy":11872177,"alias":"Lillie","name":"LillieLineman","user-since":datetime("2009-09-28T02:48:03"),"user-since-copy":datetime("2009-09-28T02:48:03"),"friend-ids":{{16078664,22307944,21464886,40255882,39090292,32823112,5748916,46831442,25498280,268782,22829744,17001614}},"employment":[{"organization-name":"Hexsanhex","start-date":date("2006-02-18")}]}
+{"id":9267397,"id-copy":9267397,"alias":"Corbin","name":"CorbinWhite","user-since":datetime("2006-01-07T07:43:27"),"user-since-copy":datetime("2006-01-07T07:43:27"),"friend-ids":{{11772390,16826538,16103166,3256508,40044263,44187580,29521314,46200384,40192445,1239869,14257012,21632509,6292478,38738535,18136574,8369661,45672754}},"employment":[{"organization-name":"Striptaxon","start-date":date("2000-09-16"),"end-date":date("2003-07-12")}]}
+{"id":9112336,"id-copy":9112336,"alias":"Marlin","name":"MarlinRosenstiehl","user-since":datetime("2010-09-26T04:27:50"),"user-since-copy":datetime("2010-09-26T04:27:50"),"friend-ids":{{10225686,16259250,11552542,28661586,8900635,27988260}},"employment":[{"organization-name":"geomedia","start-date":date("2011-12-05")}]}
+{"id":10241767,"id-copy":10241767,"alias":"Lewin","name":"LewinBurkett","user-since":datetime("2008-03-24T21:09:05"),"user-since-copy":datetime("2008-03-24T21:09:05"),"friend-ids":{{5503,32598090,36950887,22362781,16089120,30220805,6197105,44773004,17924848,36033966,41338779,38304288,18528858,6384026,46633327,18024168,13983021,7158391,31922078,1082072}},"employment":[{"organization-name":"Salthex","start-date":date("2004-02-17")}]}
+{"id":11027953,"id-copy":11027953,"alias":"Angelika","name":"AngelikaSanner","user-since":datetime("2010-10-07T04:25:19"),"user-since-copy":datetime("2010-10-07T04:25:19"),"friend-ids":{{42662440,6358862,21758734,28882210,28157558,39027509,19068795,45387055,34737892,32277859,44713546,24617807,31067294,12307376,28568916,31114183,13997610,15405045,33587810,32517419,13452101,8309328}},"employment":[{"organization-name":"Vaiatech","start-date":date("2006-02-25")}]}
+{"id":10114891,"id-copy":10114891,"alias":"Destinee","name":"DestineeLeech","user-since":datetime("2006-06-05T09:32:17"),"user-since-copy":datetime("2006-06-05T09:32:17"),"friend-ids":{{9925448,28685906,3305693,11131758,10477741,19058196,25921997,38543939,20851041}},"employment":[{"organization-name":"goldendexon","start-date":date("2001-09-24")}]}
+{"id":10177300,"id-copy":10177300,"alias":"Chase","name":"ChaseKnapp","user-since":datetime("2005-09-27T16:41:30"),"user-since-copy":datetime("2005-09-27T16:41:30"),"friend-ids":{{12805247,6093464,39416190,35877238,26583227,37835412,46337730,18107636,43948720,21031949,11688759,13980476,25486392,20775628}},"employment":[{"organization-name":"Xx-technology","start-date":date("2006-03-07"),"end-date":date("2006-05-09")}]}
+{"id":10794448,"id-copy":10794448,"alias":"Delmar","name":"DelmarDowning","user-since":datetime("2012-03-10T23:41:49"),"user-since-copy":datetime("2012-03-10T23:41:49"),"friend-ids":{{34002211,41487,45067426,9754093,23041928,41378740,4013550,11584362,46202858,43273004,35465505}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2005-09-12")}]}
+{"id":10902649,"id-copy":10902649,"alias":"Makenzie","name":"MakenzieWerner","user-since":datetime("2005-12-20T00:23:45"),"user-since-copy":datetime("2005-12-20T00:23:45"),"friend-ids":{{9011568,38173487,45649445,11873586}},"employment":[{"organization-name":"subtam","start-date":date("2000-01-06"),"end-date":date("2009-03-24")}]}
+{"id":11755633,"id-copy":11755633,"alias":"Amina","name":"AminaBurkett","user-since":datetime("2012-03-22T02:05:59"),"user-since-copy":datetime("2012-03-22T02:05:59"),"friend-ids":{{18177270,40223354,29458819,37905784,43047863,2679271,9768971,32443429,37829920,35493852,28086857,11910843,31003179,40873211,42786132,44388462}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2009-11-21"),"end-date":date("2011-03-16")}]}
+{"id":10892830,"id-copy":10892830,"alias":"Audrie","name":"AudrieHawkins","user-since":datetime("2011-11-19T00:51:33"),"user-since-copy":datetime("2011-11-19T00:51:33"),"friend-ids":{{8838768,18321840,16958648,27000957,19090823,11772058,18573458,24662627,27415154,4998699,44522833,44994903,6514403,43833807,38512495,6964420,11334788,14298721,25316052,11632302}},"employment":[{"organization-name":"Fax-fax","start-date":date("2005-02-04")}]}
+{"id":11139106,"id-copy":11139106,"alias":"Faith","name":"FaithHicks","user-since":datetime("2008-01-08T07:44:36"),"user-since-copy":datetime("2008-01-08T07:44:36"),"friend-ids":{{5409553,11995627,30724106,17065157,29513453,38627025,34382279,36487812,4292416,19328709,42169589,18029462,20202054,8738011,18339448,2522742,35366856,10669527,44287935,47124982,25912125,38893810,42212137,22227146}},"employment":[{"organization-name":"Tranzap","start-date":date("2000-11-15"),"end-date":date("2002-10-01")}]}
+{"id":9077020,"id-copy":9077020,"alias":"Marquis","name":"MarquisBunten","user-since":datetime("2008-08-23T04:31:07"),"user-since-copy":datetime("2008-08-23T04:31:07"),"friend-ids":{{16894897,21101342,27872737,14878739,47969914,38986368,20779589,4491084,21066166,40159242,25290828,43152855,41928030,2282944}},"employment":[{"organization-name":"jaydax","start-date":date("2001-07-16")}]}
+{"id":11117371,"id-copy":11117371,"alias":"Jules","name":"JulesRichardson","user-since":datetime("2009-12-06T06:21:58"),"user-since-copy":datetime("2009-12-06T06:21:58"),"friend-ids":{{75701,18653454,5088871,20583891,46460448,19742484,2433030,30869605,9273775,6556358}},"employment":[{"organization-name":"zoomplus","start-date":date("2001-09-17"),"end-date":date("2006-06-05")}]}
+{"id":11987626,"id-copy":11987626,"alias":"Chassidy","name":"ChassidyHector","user-since":datetime("2008-07-23T16:16:55"),"user-since-copy":datetime("2008-07-23T16:16:55"),"friend-ids":{{29831103,12411598,20670552,42569662}},"employment":[{"organization-name":"Y-geohex","start-date":date("2010-08-22")}]}
+{"id":9747652,"id-copy":9747652,"alias":"Graham","name":"GrahamGarratt","user-since":datetime("2006-04-16T19:35:33"),"user-since-copy":datetime("2006-04-16T19:35:33"),"friend-ids":{{9995821,7082678,29813051,33625501,32785793,23170533,26581328,35564866,9147486,17626916,12721534,22070579,25749282,27771492,35217137,6426437,4217778,6819045,6410966,43080321,32112201,20323505}},"employment":[{"organization-name":"geomedia","start-date":date("2005-09-26")}]}
+{"id":10833472,"id-copy":10833472,"alias":"Monica","name":"MonicaRyals","user-since":datetime("2009-02-14T18:52:57"),"user-since-copy":datetime("2009-02-14T18:52:57"),"friend-ids":{{34417058,24053823,28067368,16205470,24168710,9064471}},"employment":[{"organization-name":"Villa-tech","start-date":date("2003-12-03"),"end-date":date("2006-03-07")}]}
+{"id":11596522,"id-copy":11596522,"alias":"Gena","name":"GenaTurzanski","user-since":datetime("2012-06-22T18:42:25"),"user-since-copy":datetime("2012-06-22T18:42:25"),"friend-ids":{{22525625,22327219,18520174,38679685,16561552,1999972,8066310,24245231,11682156,31330371,38780021,46833789,6710024,38963740,38984150,33451484,19022059,36880540,40003274}},"employment":[{"organization-name":"Ganjastrip","start-date":date("2002-09-10")}]}
+{"id":10701727,"id-copy":10701727,"alias":"Paulita","name":"PaulitaHays","user-since":datetime("2009-11-15T15:25:08"),"user-since-copy":datetime("2009-11-15T15:25:08"),"friend-ids":{{31869253,13336594,19116516,30920596}},"employment":[{"organization-name":"Kongreen","start-date":date("2001-12-10")}]}
+{"id":9986206,"id-copy":9986206,"alias":"Tatiana","name":"TatianaAlbright","user-since":datetime("2006-03-21T10:00:55"),"user-since-copy":datetime("2006-03-21T10:00:55"),"friend-ids":{{42869099,40178170,13922993,28844962,26206785,41293581,17131809,1583964,47236558,2656158,11008100,3994698,23764118,14275676,4922979,28466879,16454954,3620561,42044685,12665882,18354684}},"employment":[{"organization-name":"Tripplelane","start-date":date("2012-04-24")}]}
+{"id":9341965,"id-copy":9341965,"alias":"Stephania","name":"StephaniaBriner","user-since":datetime("2007-06-15T18:17:32"),"user-since-copy":datetime("2007-06-15T18:17:32"),"friend-ids":{{9361850,12128362,42864061,6323327,34867192,32746507,17493376,17276666,33869929,20708786}},"employment":[{"organization-name":"Villa-tech","start-date":date("2004-03-23"),"end-date":date("2009-01-07")}]}
+{"id":11914129,"id-copy":11914129,"alias":"Ebenezer","name":"EbenezerMonahan","user-since":datetime("2006-01-08T08:17:51"),"user-since-copy":datetime("2006-01-08T08:17:51"),"friend-ids":{{9692770}},"employment":[{"organization-name":"Hatcom","start-date":date("2002-10-22"),"end-date":date("2005-07-17")}]}
+{"id":10247557,"id-copy":10247557,"alias":"Shanita","name":"ShanitaReed","user-since":datetime("2006-08-01T23:58:30"),"user-since-copy":datetime("2006-08-01T23:58:30"),"friend-ids":{{39665727,7906210,46234266,15304695,4362978,43689749,11688287,11377882,33955818,29447417,23667673,7373357,45056089,34964516,13871603,41976105,10661879,11112019,17797460}},"employment":[{"organization-name":"whitestreet","start-date":date("2004-12-08"),"end-date":date("2005-04-04")}]}
+{"id":11969527,"id-copy":11969527,"alias":"Adrian","name":"AdrianTedrow","user-since":datetime("2012-02-13T21:27:48"),"user-since-copy":datetime("2012-02-13T21:27:48"),"friend-ids":{{36940614,29564878}},"employment":[{"organization-name":"Ganjastrip","start-date":date("2002-01-16")}]}
+{"id":9373726,"id-copy":9373726,"alias":"Joe","name":"JoeRoche","user-since":datetime("2005-07-09T16:42:53"),"user-since-copy":datetime("2005-07-09T16:42:53"),"friend-ids":{{16433644,5532847,743901,2134179,43053028,36961668,9731766,45686582,17084459,27026683,1687547,6582422,38798685,9871595,2677099,42280963,32191501,4347234}},"employment":[{"organization-name":"geomedia","start-date":date("2009-09-16")}]}
+{"id":9549610,"id-copy":9549610,"alias":"Blossom","name":"BlossomGreif","user-since":datetime("2010-05-03T21:08:56"),"user-since-copy":datetime("2010-05-03T21:08:56"),"friend-ids":{{47791115,42952282}},"employment":[{"organization-name":"Solophase","start-date":date("2011-12-25"),"end-date":date("2011-11-27")}]}
+{"id":11423752,"id-copy":11423752,"alias":"Eliott","name":"EliottRoche","user-since":datetime("2007-07-01T04:36:16"),"user-since-copy":datetime("2007-07-01T04:36:16"),"friend-ids":{{34273508,10643569,13667612,19808579,46658485,43209365,7962014,24567991,25086057}},"employment":[{"organization-name":"Labzatron","start-date":date("2005-11-26")}]}
+{"id":9271291,"id-copy":9271291,"alias":"Kaitlynn","name":"KaitlynnPycroft","user-since":datetime("2010-10-09T11:30:12"),"user-since-copy":datetime("2010-10-09T11:30:12"),"friend-ids":{{38067939,25732262,17076819,19477302,29794559}},"employment":[{"organization-name":"Techitechi","start-date":date("2006-09-04")}]}
+{"id":11147050,"id-copy":11147050,"alias":"Karena","name":"KarenaTanner","user-since":datetime("2007-03-17T08:50:48"),"user-since-copy":datetime("2007-03-17T08:50:48"),"friend-ids":{{39952587,2518830,30305705,21365609,45914603,2590495,8595660}},"employment":[{"organization-name":"freshdox","start-date":date("2000-11-13"),"end-date":date("2009-01-10")}]}
+{"id":11347261,"id-copy":11347261,"alias":"Linda","name":"LindaBaldwin","user-since":datetime("2010-04-21T08:05:44"),"user-since-copy":datetime("2010-04-21T08:05:44"),"friend-ids":{{1423464,7534626,19522889,25132532,19933077,36713596,31725151,46644015,17758352,37356325,43714985,29437022,21616894,32487769,18527683,32632034,5598064,47187635,23490346}},"employment":[{"organization-name":"overtech","start-date":date("2005-06-22"),"end-date":date("2007-02-18")}]}
+{"id":10943026,"id-copy":10943026,"alias":"Raeburn","name":"RaeburnAllshouse","user-since":datetime("2008-08-26T04:51:27"),"user-since-copy":datetime("2008-08-26T04:51:27"),"friend-ids":{{6784667,1651647,45052591,21630976,20049039,37839759,38694475,23340828,8641638,4568782,35684305,20895609,2213341,8612199,14260231,8621325,21926952,41656664,45180955}},"employment":[{"organization-name":"Doublezone","start-date":date("2007-09-28")}]}
+{"id":9205615,"id-copy":9205615,"alias":"Eddie","name":"EddieRosensteel","user-since":datetime("2007-01-03T07:17:37"),"user-since-copy":datetime("2007-01-03T07:17:37"),"friend-ids":{{4208455,19941893}},"employment":[{"organization-name":"zoomplus","start-date":date("2005-11-15")}]}
+{"id":11318329,"id-copy":11318329,"alias":"April","name":"AprilSurrency","user-since":datetime("2008-09-02T21:07:03"),"user-since-copy":datetime("2008-09-02T21:07:03"),"friend-ids":{{8646916,27873471,41336682,42549624,39851926,29548550,31209458,40169445,27695329,20395537,10311481,47078664,32368262,6850643,26890752}},"employment":[{"organization-name":"Newcom","start-date":date("2009-12-11")}]}
+{"id":11269867,"id-copy":11269867,"alias":"Bettye","name":"BettyeTeagarden","user-since":datetime("2006-02-15T08:28:04"),"user-since-copy":datetime("2006-02-15T08:28:04"),"friend-ids":{{3227122,9086278,26175058,16380287,15179776,6343969,15198730,7420831,38504400,5337815,35914644,42885098,2521174,43359140,17884442,3131060,35723204,14956242,78003,7455524,3371831,46465463,9947087}},"employment":[{"organization-name":"Quadlane","start-date":date("2000-07-21"),"end-date":date("2007-10-28")}]}
+{"id":11830822,"id-copy":11830822,"alias":"Lincoln","name":"LincolnFuchs","user-since":datetime("2008-01-22T19:08:51"),"user-since-copy":datetime("2008-01-22T19:08:51"),"friend-ids":{{29014579,29789039,2225447,37872940,37026231,3223799,40601178}},"employment":[{"organization-name":"Villa-tech","start-date":date("2006-01-14"),"end-date":date("2010-04-24")}]}
+{"id":9886819,"id-copy":9886819,"alias":"Phoebe","name":"PhoebeBarnes","user-since":datetime("2010-12-26T07:30:15"),"user-since-copy":datetime("2010-12-26T07:30:15"),"friend-ids":{{24361962,43750816,46566991,4790101,38827567,6893116,41555542,35877264,18479056,22186674,10954414,43453344,11903159,12257863,45299776}},"employment":[{"organization-name":"Sanjodax","start-date":date("2000-01-02"),"end-date":date("2008-05-24")}]}
+{"id":9525361,"id-copy":9525361,"alias":"Leonardo","name":"LeonardoSurrency","user-since":datetime("2008-12-21T10:09:26"),"user-since-copy":datetime("2008-12-21T10:09:26"),"friend-ids":{{12471014,47714763,18071069,32545366,46041462,35261185,20826834,29002678,47207065,7370034,38283272,47090645,33425043,16014552,15633873,24101778,26168621,21955493,17856723,18158610}},"employment":[{"organization-name":"Keytech","start-date":date("2011-12-06"),"end-date":date("2011-04-04")}]}
+{"id":9440452,"id-copy":9440452,"alias":"Maria","name":"MariaField","user-since":datetime("2010-04-06T15:15:24"),"user-since-copy":datetime("2010-04-06T15:15:24"),"friend-ids":{{35137543,24166956,45255343,10050289,27769291,40368984,38146662,43123957,10442976,46931482,447566,14148069,39035817,32169234,35607837,8648749,3741547,31840808,3029722,40917859}},"employment":[{"organization-name":"zoomplus","start-date":date("2012-04-27"),"end-date":date("2012-05-11")}]}
+{"id":11525575,"id-copy":11525575,"alias":"Zack","name":"ZackMills","user-since":datetime("2007-10-15T20:53:30"),"user-since-copy":datetime("2007-10-15T20:53:30"),"friend-ids":{{11119738,47490530,18951399,24413247,4019030,39064308,43279140,11316225,15383674,40613636,4793869,21591307,23561981,3763992,32892218,34334911,40693733}},"employment":[{"organization-name":"Rungozoom","start-date":date("2012-05-25"),"end-date":date("2012-07-09")}]}
+{"id":10047001,"id-copy":10047001,"alias":"Darcy","name":"DarcyKava","user-since":datetime("2012-02-25T17:16:18"),"user-since-copy":datetime("2012-02-25T17:16:18"),"friend-ids":{{15613341,46557569,20439965,22442508,32423739,40757483,36365324,40706148,12537361,47741886,24508947,34168899,10674474,34285157,28222068,11113263}},"employment":[{"organization-name":"kin-ron","start-date":date("2008-01-27")}]}
+{"id":11595592,"id-copy":11595592,"alias":"Bert","name":"BertAtkinson","user-since":datetime("2011-09-03T07:24:42"),"user-since-copy":datetime("2011-09-03T07:24:42"),"friend-ids":{{36724561,45824456,33567747,21400268,11419574,47463040,6480088,45216774,26857982,7140352,1884512,29610211,2626672,41371388,43582371,42445087,14734124,3580372,40134022}},"employment":[{"organization-name":"Keytech","start-date":date("2006-06-27"),"end-date":date("2007-06-07")}]}
+{"id":11330215,"id-copy":11330215,"alias":"Tilly","name":"TillyMckinnon","user-since":datetime("2011-04-13T10:13:13"),"user-since-copy":datetime("2011-04-13T10:13:13"),"friend-ids":{{5559510,31907101,45791333,35002065,1302921,37193818,32812039,41322357,20631502}},"employment":[{"organization-name":"Groovetex","start-date":date("2001-03-22"),"end-date":date("2008-08-22")}]}
+{"id":11893462,"id-copy":11893462,"alias":"Shonna","name":"ShonnaDickson","user-since":datetime("2007-06-12T09:36:50"),"user-since-copy":datetime("2007-06-12T09:36:50"),"friend-ids":{{30462288,43630666,35884473,25217438,3196051,41844836,8922622,15388786,33486563,22739607,42411271,47936046,8921955,11314832,13138669,1057389,45874085}},"employment":[{"organization-name":"Zuncan","start-date":date("2010-06-18")}]}
+{"id":10317160,"id-copy":10317160,"alias":"Maria","name":"MariaHair","user-since":datetime("2006-05-21T16:06:00"),"user-since-copy":datetime("2006-05-21T16:06:00"),"friend-ids":{{7063473,43027344,2119671,39231388,34041933,5141408,20278936}},"employment":[{"organization-name":"Sancone","start-date":date("2005-10-20")}]}
+{"id":10554112,"id-copy":10554112,"alias":"Virgil","name":"VirgilBickerson","user-since":datetime("2006-03-14T07:07:42"),"user-since-copy":datetime("2006-03-14T07:07:42"),"friend-ids":{{21584501,3506050,31062036,20425233,6548274,12613206,16607156}},"employment":[{"organization-name":"Viatechi","start-date":date("2004-08-25"),"end-date":date("2006-11-11")}]}
+{"id":9420304,"id-copy":9420304,"alias":"Alwyn","name":"AlwynAkers","user-since":datetime("2009-11-08T08:30:46"),"user-since-copy":datetime("2009-11-08T08:30:46"),"friend-ids":{{40384671,13399303,2163402}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2012-06-14"),"end-date":date("2012-07-17")}]}
+{"id":10269739,"id-copy":10269739,"alias":"Shantel","name":"ShantelEve","user-since":datetime("2012-06-06T00:37:05"),"user-since-copy":datetime("2012-06-06T00:37:05"),"friend-ids":{{39436396,20382971,47821933,28867521,23217564,40672635,34693766,4383592,42534606,23535312,9112260,4828073,37429286,27965200,30257544,47609429,18527025,33339218,898986,2817270,6040099,47802547}},"employment":[{"organization-name":"Sublamdox","start-date":date("2000-05-24")}]}
+{"id":10089976,"id-copy":10089976,"alias":"Marion","name":"MarionThomlinson","user-since":datetime("2006-06-27T14:11:49"),"user-since-copy":datetime("2006-06-27T14:11:49"),"friend-ids":{{39404598,46190974,43413339,41250692,4194349,5150083,35574492,30896673,15969653,41889132,38801872,17834003,42587459,42269051,20206793,46257713,2735409,28567746,6641216,3627253,15945805,33861471,9997931,38242090}},"employment":[{"organization-name":"Sanjodax","start-date":date("2011-11-22"),"end-date":date("2011-06-01")}]}
+{"id":11203174,"id-copy":11203174,"alias":"Lise","name":"LiseRockwell","user-since":datetime("2005-04-21T02:17:33"),"user-since-copy":datetime("2005-04-21T02:17:33"),"friend-ids":{{25322984,687106,15193641,24397137,34772763,24725595,30853266,14933558,36895249,39451299,2620397,44594032,3455415,39921033,21621070,800967}},"employment":[{"organization-name":"Redelectronics","start-date":date("2006-01-13"),"end-date":date("2008-07-23")}]}
+{"id":11051014,"id-copy":11051014,"alias":"Tad","name":"TadWilson","user-since":datetime("2011-05-05T14:48:34"),"user-since-copy":datetime("2011-05-05T14:48:34"),"friend-ids":{{42862096,17517240,8058482,9927174,4207109,4924943,11531213}},"employment":[{"organization-name":"Canline","start-date":date("2005-01-25"),"end-date":date("2010-11-14")}]}
+{"id":9379330,"id-copy":9379330,"alias":"Esther","name":"EstherReichard","user-since":datetime("2006-09-23T09:53:43"),"user-since-copy":datetime("2006-09-23T09:53:43"),"friend-ids":{{29035495,33601969,32342695,28995226,34638799,38330225,38512256}},"employment":[{"organization-name":"Ronholdings","start-date":date("2006-05-27")}]}
+{"id":9719995,"id-copy":9719995,"alias":"Hazel","name":"HazelKnopsnider","user-since":datetime("2007-04-05T01:11:42"),"user-since-copy":datetime("2007-04-05T01:11:42"),"friend-ids":{{38515770,23212874,6000594,27957554,28093880,3726628,22800428,42313894,23190476,18537188,22083915,43478674,33364444,19158958,1590605,36792931,42057988,33286729,29580197,25232028}},"employment":[{"organization-name":"Y-geohex","start-date":date("2008-09-28")}]}
+{"id":11559262,"id-copy":11559262,"alias":"Herb","name":"HerbPaul","user-since":datetime("2011-04-09T22:23:26"),"user-since-copy":datetime("2011-04-09T22:23:26"),"friend-ids":{{46915837,26659094}},"employment":[{"organization-name":"Qvohouse","start-date":date("2011-07-05"),"end-date":date("2011-07-07")}]}
+{"id":11262439,"id-copy":11262439,"alias":"Alexandra","name":"AlexandraStocker","user-since":datetime("2010-08-28T03:48:52"),"user-since-copy":datetime("2010-08-28T03:48:52"),"friend-ids":{{16331707}},"employment":[{"organization-name":"zoomplus","start-date":date("2010-12-04")}]}
+{"id":9136882,"id-copy":9136882,"alias":"Cassie","name":"CassieGarratt","user-since":datetime("2005-08-07T05:09:11"),"user-since-copy":datetime("2005-08-07T05:09:11"),"friend-ids":{{40916371,42882703,37748113,45347468,37653228,15540626,29276950,31566687,14600173,12909057,39561446,41035377,45987458,43649639,24488758,25625568,15566464,584815,35900688,1079087,46148561,46404398}},"employment":[{"organization-name":"U-electrics","start-date":date("2010-11-13"),"end-date":date("2010-09-04")}]}
+{"id":10230604,"id-copy":10230604,"alias":"Courtney","name":"CourtneyCountryman","user-since":datetime("2012-03-05T08:49:56"),"user-since-copy":datetime("2012-03-05T08:49:56"),"friend-ids":{{28617094,31170285,26700577,43586990,12809105,8131401,15644912,38127923,7871621,13276397,41863539,3715524,13404150,12834697,237361,41295097,29471386,19859329,14312407,79917,42547367,9661712,30110962,29137807}},"employment":[{"organization-name":"Unijobam","start-date":date("2001-06-09"),"end-date":date("2004-06-04")}]}
+{"id":9067279,"id-copy":9067279,"alias":"Jeanine","name":"JeanineEmrick","user-since":datetime("2011-06-25T09:43:07"),"user-since-copy":datetime("2011-06-25T09:43:07"),"friend-ids":{{12884712,45104617,41134568,15844605,645264,33182092,16884335,46281324,3977219,5682848,441588,26025738,3165091,21821928,23073877}},"employment":[{"organization-name":"Streettax","start-date":date("2012-01-02")}]}
+{"id":9822973,"id-copy":9822973,"alias":"Melia","name":"MeliaWentzel","user-since":datetime("2012-07-17T05:10:30"),"user-since-copy":datetime("2012-07-17T05:10:30"),"friend-ids":{{2563633,27918474,42233962,40497985,4437912,43013491,47283180,20434605,25309336,11299381,20584869,15093618,14273412,46920368,5868827,40191100,44286983,11787568,44551406}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2001-07-07")}]}
+{"id":11481961,"id-copy":11481961,"alias":"Ralph","name":"RalphMinnie","user-since":datetime("2008-09-03T03:36:09"),"user-since-copy":datetime("2008-09-03T03:36:09"),"friend-ids":{{28795092,15427393,13323116,6103928,22507606,38931008,8419762,30922606,11217439,41769747,19668638,26796252,26750627,4855539,11170229,30124829,16596482,15728547,46139530,43784722,20640234,22313927,16136087,39688415}},"employment":[{"organization-name":"Doncare","start-date":date("2006-02-13")}]}
+{"id":9934939,"id-copy":9934939,"alias":"Camilla","name":"CamillaRhinehart","user-since":datetime("2008-12-06T10:44:45"),"user-since-copy":datetime("2008-12-06T10:44:45"),"friend-ids":{{17020237,36188716,32765819,20068359,23060675,16692600}},"employment":[{"organization-name":"Ransaofan","start-date":date("2012-04-05")}]}
+{"id":9635563,"id-copy":9635563,"alias":"Tamsen","name":"TamsenCowart","user-since":datetime("2010-10-07T05:11:20"),"user-since-copy":datetime("2010-10-07T05:11:20"),"friend-ids":{{}},"employment":[{"organization-name":"Tanzumbam","start-date":date("2007-01-07")}]}
+{"id":10540825,"id-copy":10540825,"alias":"Jayna","name":"JaynaRowe","user-since":datetime("2008-01-09T12:09:19"),"user-since-copy":datetime("2008-01-09T12:09:19"),"friend-ids":{{20315422,9358699,6204561,40594838,46678685,34224970,47262413,42477325,7591560,39986319,9438124,30292072,11187685,27885,47428887,9535830,36979072,14613793}},"employment":[{"organization-name":"Kanelectrics","start-date":date("2005-11-10")}]}
+{"id":10911220,"id-copy":10911220,"alias":"Laurice","name":"LauriceDuncan","user-since":datetime("2008-08-05T15:55:34"),"user-since-copy":datetime("2008-08-05T15:55:34"),"friend-ids":{{212109}},"employment":[{"organization-name":"Qvohouse","start-date":date("2001-02-03")}]}
+{"id":11793622,"id-copy":11793622,"alias":"Leonard","name":"LeonardAlice","user-since":datetime("2011-03-02T21:42:07"),"user-since-copy":datetime("2011-03-02T21:42:07"),"friend-ids":{{38648452,2302677,713863,2484976,20706899,6649310,9952945,1293945,23188221,43521816,2398744,28382427,45933146,27717079,12894240,8077643,38945982,12658937,36047491,42431984,43626155}},"employment":[{"organization-name":"Newphase","start-date":date("2001-02-12"),"end-date":date("2001-06-02")}]}
+{"id":10580422,"id-copy":10580422,"alias":"Travers","name":"TraversSadley","user-since":datetime("2011-02-09T08:22:49"),"user-since-copy":datetime("2011-02-09T08:22:49"),"friend-ids":{{36067992,8651663,43180149,732576,35709545,30999437}},"employment":[{"organization-name":"Villa-dox","start-date":date("2009-05-27")}]}
+{"id":10729942,"id-copy":10729942,"alias":"Valda","name":"ValdaFea","user-since":datetime("2005-07-16T09:31:53"),"user-since-copy":datetime("2005-07-16T09:31:53"),"friend-ids":{{20145015,42027050,38819467,3406065,4977132,47154979,23685067}},"employment":[{"organization-name":"Greencare","start-date":date("2008-10-12")}]}
+{"id":9699673,"id-copy":9699673,"alias":"Jim","name":"JimPycroft","user-since":datetime("2012-07-25T20:20:38"),"user-since-copy":datetime("2012-07-25T20:20:38"),"friend-ids":{{14858146,47543880,3186927,38198580,2365336,5255886,11178580,41188272,17623582,6422949,4405751,12128017,32409443,38861849,16511892,24515731,46665640,40644816,19341995,44288533,26148671}},"employment":[{"organization-name":"Codetechno","start-date":date("2007-01-24"),"end-date":date("2009-12-16")}]}
+{"id":9233794,"id-copy":9233794,"alias":"Jeffrey","name":"JeffreyThrockmorton","user-since":datetime("2005-04-23T04:24:31"),"user-since-copy":datetime("2005-04-23T04:24:31"),"friend-ids":{{29565308,29107229,35495609,27358360,24507795,18583779,16799427,3571959,6539875,32120867,17248402,12227155,37995559,29425657,20855502}},"employment":[{"organization-name":"Fax-fax","start-date":date("2000-04-22"),"end-date":date("2010-05-28")}]}
+{"id":9643768,"id-copy":9643768,"alias":"Gil","name":"GilVeith","user-since":datetime("2006-04-26T11:42:30"),"user-since-copy":datetime("2006-04-26T11:42:30"),"friend-ids":{{22270431,9614818,9080111,6500797,37876717,28122656,13971193,20936637,19883735,37455193,32129291,40710966,17779823,41523128,41276564,34424817,19326867,26058281}},"employment":[{"organization-name":"Xx-technology","start-date":date("2002-02-04")}]}
+{"id":11637820,"id-copy":11637820,"alias":"Aislin","name":"AislinPyle","user-since":datetime("2005-01-04T00:11:51"),"user-since-copy":datetime("2005-01-04T00:11:51"),"friend-ids":{{17232277,46376966,22503632,14771156,37550654,3930020,7116826,38303815,30210948,10532544,44382464,32051602}},"employment":[{"organization-name":"Mathtech","start-date":date("2004-05-06")}]}
+{"id":11921524,"id-copy":11921524,"alias":"Mickey","name":"MickeySybilla","user-since":datetime("2012-03-28T17:05:25"),"user-since-copy":datetime("2012-03-28T17:05:25"),"friend-ids":{{40813978,14172552,40702786,929262,2220334,33077762,20716547,11400385,21916926,38422356,13378381,32362984,8162369,8965084,37823302,3542211,29294304,37672739,28359647}},"employment":[{"organization-name":"Solfix","start-date":date("2007-09-27")}]}
+{"id":10133458,"id-copy":10133458,"alias":"Kati","name":"KatiPennington","user-since":datetime("2011-01-28T10:51:37"),"user-since-copy":datetime("2011-01-28T10:51:37"),"friend-ids":{{41299906,11523198,8344474,36086944,34330342,43585884,6751565,23415221,32275829,43645200}},"employment":[{"organization-name":"Tranzap","start-date":date("2005-11-11")}]}
+{"id":9212815,"id-copy":9212815,"alias":"Erica","name":"EricaBraun","user-since":datetime("2009-01-11T07:32:03"),"user-since-copy":datetime("2009-01-11T07:32:03"),"friend-ids":{{1314906,6581233,35117578,11133528,19606776,37833518,40040803,44107209,38804989,35779440,41138709}},"employment":[{"organization-name":"Doublezone","start-date":date("2008-04-03")}]}
+{"id":9879709,"id-copy":9879709,"alias":"Winfred","name":"WinfredCraig","user-since":datetime("2005-08-03T19:34:00"),"user-since-copy":datetime("2005-08-03T19:34:00"),"friend-ids":{{22314477,25116324,22136373,35942614,21324680,17967388,29463891,36125380,20673052,27353154,25107580,24689990,17672337,16922511,26158336,35966438,26619840,29808016,12075922,33292381,17902188}},"employment":[{"organization-name":"Trustbam","start-date":date("2010-02-04")}]}
+{"id":9804973,"id-copy":9804973,"alias":"Harriette","name":"HarrietteHoopengarner","user-since":datetime("2011-08-14T20:51:52"),"user-since-copy":datetime("2011-08-14T20:51:52"),"friend-ids":{{18754696,27799194,36904141,29647419,8521621,35146470,45194388,43397176,12596887,33315,39826335,31228413,123596,35927645,11445687,33208186,21941268}},"employment":[{"organization-name":"Doublezone","start-date":date("2003-03-24")}]}
+{"id":10690066,"id-copy":10690066,"alias":"Abraham","name":"AbrahamWardle","user-since":datetime("2006-04-08T20:27:10"),"user-since-copy":datetime("2006-04-08T20:27:10"),"friend-ids":{{18105973,39839261,27532181,2565949,37077592,28929530}},"employment":[{"organization-name":"Hatcom","start-date":date("2005-06-03"),"end-date":date("2006-12-02")}]}
+{"id":11064301,"id-copy":11064301,"alias":"Dave","name":"DaveNicholas","user-since":datetime("2007-01-09T09:19:57"),"user-since-copy":datetime("2007-01-09T09:19:57"),"friend-ids":{{19136340,40809808,18774928,405329,27436466,35586548,16671212,44582715,47932437,22599645,26281489,39246487,39088455,43696576,28175190}},"employment":[{"organization-name":"geomedia","start-date":date("2005-02-04")}]}
+{"id":9814867,"id-copy":9814867,"alias":"Pacey","name":"PaceyBranson","user-since":datetime("2011-07-05T06:49:42"),"user-since-copy":datetime("2011-07-05T06:49:42"),"friend-ids":{{7196953}},"employment":[{"organization-name":"linedexon","start-date":date("2005-11-19"),"end-date":date("2007-12-03")}]}
+{"id":10166767,"id-copy":10166767,"alias":"Leon","name":"LeonWardle","user-since":datetime("2008-05-19T07:05:45"),"user-since-copy":datetime("2008-05-19T07:05:45"),"friend-ids":{{41883510,44504996,36617462,32609381,11246739,18717645,32225763,25136144,18258339,4951535,40063362,38810936,1994155,16613514,25411748,34221779,44135463}},"employment":[{"organization-name":"Inchex","start-date":date("2006-10-11")}]}
+{"id":11672578,"id-copy":11672578,"alias":"Juli","name":"JuliMcclymonds","user-since":datetime("2010-07-17T13:53:57"),"user-since-copy":datetime("2010-07-17T13:53:57"),"friend-ids":{{16548983,7350585,44497037}},"employment":[{"organization-name":"Groovetex","start-date":date("2003-05-23"),"end-date":date("2009-08-01")}]}
+{"id":9546133,"id-copy":9546133,"alias":"Renae","name":"RenaeWhitehead","user-since":datetime("2012-04-21T14:38:30"),"user-since-copy":datetime("2012-04-21T14:38:30"),"friend-ids":{{31261211,19892104,35568606,12050300,42512152,37032282,27185051}},"employment":[{"organization-name":"U-ron","start-date":date("2012-02-20"),"end-date":date("2012-07-04")}]}
+{"id":11087839,"id-copy":11087839,"alias":"Manfred","name":"ManfredEdwards","user-since":datetime("2009-10-01T09:12:15"),"user-since-copy":datetime("2009-10-01T09:12:15"),"friend-ids":{{7828089}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2003-07-25")}]}
+{"id":9421798,"id-copy":9421798,"alias":"Jaqueline","name":"JaquelineHasely","user-since":datetime("2011-06-06T16:32:03"),"user-since-copy":datetime("2011-06-06T16:32:03"),"friend-ids":{{17911249,45887650,15916739,42045244,42824039,4802136,43709530,41533233,13714833,33000412,29627102,43277560,3727319,19030370,47600623,27902511,13460397,34825938,9726577,10062858,34721080,6725312,21572679}},"employment":[{"organization-name":"Hatcom","start-date":date("2004-02-17")}]}
+{"id":9442978,"id-copy":9442978,"alias":"Osborne","name":"OsborneHiles","user-since":datetime("2012-07-28T10:59:39"),"user-since-copy":datetime("2012-07-28T10:59:39"),"friend-ids":{{40833026,39533118,6206868,27383373,3010465,14776443,43239645,21956253,4112089,27667721,34336067,38377619,32701403,20907262,32732275,30488150,12349697,47468946,20956164,16141416}},"employment":[{"organization-name":"Freshfix","start-date":date("2011-08-21")}]}
+{"id":11720794,"id-copy":11720794,"alias":"Alisha","name":"AlishaTue","user-since":datetime("2010-08-11T01:17:31"),"user-since-copy":datetime("2010-08-11T01:17:31"),"friend-ids":{{6380101,43972052,6557931,42465959,21268624,35831867,45839471,37781645,34750475,35886124,4491900}},"employment":[{"organization-name":"Ransaofan","start-date":date("2001-02-02")}]}
+{"id":10178518,"id-copy":10178518,"alias":"Rudyard","name":"RudyardMcmullen","user-since":datetime("2011-05-06T14:57:22"),"user-since-copy":datetime("2011-05-06T14:57:22"),"friend-ids":{{25647527,14445589,47924548,24945241,13505530,39640007,6132209,815976,31529708,28281922,17886251,42402860,18330827,13619952}},"employment":[{"organization-name":"Streettax","start-date":date("2006-06-24")}]}
+{"id":9779623,"id-copy":9779623,"alias":"Alberto","name":"AlbertoCraig","user-since":datetime("2009-11-25T14:48:04"),"user-since-copy":datetime("2009-11-25T14:48:04"),"friend-ids":{{6737836,26882597,30254391,4861442,18105612}},"employment":[{"organization-name":"Goldcity","start-date":date("2012-01-25")}]}
+{"id":10532791,"id-copy":10532791,"alias":"Byrne","name":"ByrneLafortune","user-since":datetime("2010-03-13T13:21:05"),"user-since-copy":datetime("2010-03-13T13:21:05"),"friend-ids":{{35020297,40002497,16857157,47134232,37864297,31029450,36968713,36672267,15503365,43888732,29395734,35372186,19093208,21774877,9785166,22833579}},"employment":[{"organization-name":"silfind","start-date":date("2003-12-21")}]}
+{"id":10398562,"id-copy":10398562,"alias":"Brendon","name":"BrendonMaclagan","user-since":datetime("2012-02-23T06:18:49"),"user-since-copy":datetime("2012-02-23T06:18:49"),"friend-ids":{{39206829,37980663,36889290,9114653,26448451,15142055,23349234,11668644,22072984,2091972,957976,26110137,20947598,32127830,35850034,39029675,21265582,26725192,13963111,4392994,37042547}},"employment":[{"organization-name":"Newhotplus","start-date":date("2001-09-21")}]}
+{"id":11547586,"id-copy":11547586,"alias":"Rosanne","name":"RosanneWatkins","user-since":datetime("2008-03-02T16:07:45"),"user-since-copy":datetime("2008-03-02T16:07:45"),"friend-ids":{{47389452,44553302,30722503,3892313,9603884,12058710,18459884,23971280,39791340,25400946,25149383,8391991,6548649,20662585,34505551,8352025}},"employment":[{"organization-name":"Villa-dox","start-date":date("2005-05-17")}]}
+{"id":11220541,"id-copy":11220541,"alias":"Phyllida","name":"PhyllidaRing","user-since":datetime("2012-03-01T06:11:58"),"user-since-copy":datetime("2012-03-01T06:11:58"),"friend-ids":{{609357,45820919,17439004,16790980,27878958,13930012,20759108,23987257,29330180,9298668,10644382,2596101,29705735,13371057,41709459,6973880,41608321,41344973,9555209,37508452,26445359,7693361,12059348}},"employment":[{"organization-name":"Zuncan","start-date":date("2002-12-05"),"end-date":date("2009-09-16")}]}
+{"id":10894411,"id-copy":10894411,"alias":"Lacy","name":"LacyShaw","user-since":datetime("2006-04-06T00:11:24"),"user-since-copy":datetime("2006-04-06T00:11:24"),"friend-ids":{{4203591,28370134,5239468,12951448,39355113,9126812,5662652,4633221,11954172,33269236,11545355,14018236,21980886,34750979,22877356}},"employment":[{"organization-name":"Quoline","start-date":date("2009-04-12")}]}
+{"id":10989949,"id-copy":10989949,"alias":"Kaylyn","name":"KaylynElder","user-since":datetime("2011-01-13T12:02:13"),"user-since-copy":datetime("2011-01-13T12:02:13"),"friend-ids":{{22698118,31639011,11500577,13007617,26781164,20827141,9916306,26415081,14027605,19305199,45276489,17632806,42243983}},"employment":[{"organization-name":"Ontotanin","start-date":date("2006-01-05")}]}
+{"id":10261300,"id-copy":10261300,"alias":"Nick","name":"NickRohtin","user-since":datetime("2007-01-24T17:56:52"),"user-since-copy":datetime("2007-01-24T17:56:52"),"friend-ids":{{37649902}},"employment":[{"organization-name":"Solophase","start-date":date("2004-03-06"),"end-date":date("2007-05-20")}]}
+{"id":10851595,"id-copy":10851595,"alias":"Juan","name":"JuanSoames","user-since":datetime("2006-02-16T05:34:28"),"user-since-copy":datetime("2006-02-16T05:34:28"),"friend-ids":{{34589906,8801547,38357163,39649840,18254469,38911658,17825991,26015024,29742264,13155934,28459597,34931012,20376527}},"employment":[{"organization-name":"Newhotplus","start-date":date("2008-11-17"),"end-date":date("2009-01-13")}]}
+{"id":11494930,"id-copy":11494930,"alias":"Eleanor","name":"EleanorAnderson","user-since":datetime("2008-09-01T04:27:31"),"user-since-copy":datetime("2008-09-01T04:27:31"),"friend-ids":{{46834294,32081711}},"employment":[{"organization-name":"Sanjodax","start-date":date("2008-01-19")}]}
+{"id":9931588,"id-copy":9931588,"alias":"Sheri","name":"SheriHindman","user-since":datetime("2011-02-19T03:55:37"),"user-since-copy":datetime("2011-02-19T03:55:37"),"friend-ids":{{10993709,28005344,31884585,1581885,46332238,47401902,38814902,39736365,24318394,15329318,35794552,14913021,8723328,28102869,27218765,21310255}},"employment":[{"organization-name":"Hexviafind","start-date":date("2011-08-17"),"end-date":date("2011-12-15")}]}
+{"id":10131352,"id-copy":10131352,"alias":"Brett","name":"BrettBullard","user-since":datetime("2011-03-20T00:21:15"),"user-since-copy":datetime("2011-03-20T00:21:15"),"friend-ids":{{42102691,34313392,19476509,40509353,40764048,32856149,20306336,18276288,34284082,32265145,23912229,7426729,26377621,43687843,6140857,4573908,6840657,18335864,19868141,6051525}},"employment":[{"organization-name":"Mathtech","start-date":date("2005-11-09"),"end-date":date("2008-12-05")}]}
+{"id":11953306,"id-copy":11953306,"alias":"Teale","name":"TealeHoltzer","user-since":datetime("2007-02-14T21:50:54"),"user-since-copy":datetime("2007-02-14T21:50:54"),"friend-ids":{{30902622,26223630,46832466,32585590,34005386,23371032,25984545,7502619}},"employment":[{"organization-name":"Newphase","start-date":date("2010-02-14"),"end-date":date("2011-07-08")}]}
+{"id":10486213,"id-copy":10486213,"alias":"Modesto","name":"ModestoCox","user-since":datetime("2006-02-07T05:43:24"),"user-since-copy":datetime("2006-02-07T05:43:24"),"friend-ids":{{42665859,12929499,5618502,24287766,38722882,5162913,2978226,37521984,43144325,3313029,17680751,726799}},"employment":[{"organization-name":"Newhotplus","start-date":date("2007-08-12")}]}
+{"id":10940377,"id-copy":10940377,"alias":"Lory","name":"LoryElless","user-since":datetime("2011-03-21T19:07:17"),"user-since-copy":datetime("2011-03-21T19:07:17"),"friend-ids":{{38950352,10596357,43176277,27274342,27082326}},"employment":[{"organization-name":"Technohow","start-date":date("2007-04-19")}]}
+{"id":10708477,"id-copy":10708477,"alias":"Zacharias","name":"ZachariasRandolph","user-since":datetime("2008-07-13T16:12:33"),"user-since-copy":datetime("2008-07-13T16:12:33"),"friend-ids":{{18251027,47694844,25569678,33130234,7351010,32617025,40619749,28576965,34970660,34320919,17056847,46007935,244756,3130710,5218614,6968874,19440356,448790,3336700,44725864,24738046,6159443,14380294,20289778}},"employment":[{"organization-name":"Streettax","start-date":date("2007-09-09")}]}
+{"id":11972860,"id-copy":11972860,"alias":"Isador","name":"IsadorCattley","user-since":datetime("2005-04-10T23:37:49"),"user-since-copy":datetime("2005-04-10T23:37:49"),"friend-ids":{{39841874,9405322,3110197,39455453,11331432,31809217,45852118,12899824,19561127,3413313,19872192,13427579,140732,6913603}},"employment":[{"organization-name":"freshdox","start-date":date("2006-01-01"),"end-date":date("2009-11-22")}]}
+{"id":11839117,"id-copy":11839117,"alias":"Kyra","name":"KyraMcdonald","user-since":datetime("2010-07-08T20:46:49"),"user-since-copy":datetime("2010-07-08T20:46:49"),"friend-ids":{{42933043,41665211,13075886,36147059,20127919,31449381,47427643,24399833,16541120,38909218,15609877,46802599,31772232,46743670}},"employment":[{"organization-name":"Dancode","start-date":date("2005-06-08"),"end-date":date("2007-11-11")}]}
+{"id":9332161,"id-copy":9332161,"alias":"Lavinia","name":"LaviniaLineman","user-since":datetime("2006-02-07T20:39:55"),"user-since-copy":datetime("2006-02-07T20:39:55"),"friend-ids":{{21419337,31581364}},"employment":[{"organization-name":"Sancone","start-date":date("2012-07-05")}]}
+{"id":10869727,"id-copy":10869727,"alias":"Jacquetta","name":"JacquettaMaugham","user-since":datetime("2010-07-11T22:43:19"),"user-since-copy":datetime("2010-07-11T22:43:19"),"friend-ids":{{36109878,46889968,19648550,14051620,14645938,14933447,33880415}},"employment":[{"organization-name":"Trustbam","start-date":date("2002-12-09")}]}
+{"id":11427397,"id-copy":11427397,"alias":"Oscar","name":"OscarMillhouse","user-since":datetime("2012-04-07T04:52:39"),"user-since-copy":datetime("2012-04-07T04:52:39"),"friend-ids":{{27577077,26831616,24024317,24669981,15864715,41688094,25689775,19288762,25015698,24343183,30170416,39881555,29378159,6748762,45948007}},"employment":[{"organization-name":"Plexlane","start-date":date("2012-05-15")}]}
+{"id":9872791,"id-copy":9872791,"alias":"Yasmine","name":"YasmineCanham","user-since":datetime("2005-06-08T14:45:42"),"user-since-copy":datetime("2005-06-08T14:45:42"),"friend-ids":{{7340569,16137560,43341029,31700386,24881875,17852264,42730676,32655012}},"employment":[{"organization-name":"Rungozoom","start-date":date("2004-05-09"),"end-date":date("2011-02-28")}]}
+{"id":9664990,"id-copy":9664990,"alias":"Travis","name":"TravisJube","user-since":datetime("2010-02-12T13:42:04"),"user-since-copy":datetime("2010-02-12T13:42:04"),"friend-ids":{{22627931,5992593,8208547,37326819,14939087,18366709,29043862,45062025,21360937,19730114,26779317,46856921,28406774,40580511,8062361,2179206,47765870,14039643,28857662,42600706}},"employment":[{"organization-name":"Lexitechno","start-date":date("2007-01-14")}]}
+{"id":9483769,"id-copy":9483769,"alias":"Marketta","name":"MarkettaSchere","user-since":datetime("2006-04-02T05:48:16"),"user-since-copy":datetime("2006-04-02T05:48:16"),"friend-ids":{{15151816,38432593,14501842,21508230,20201815,35434395,46212890,9387767,35469959,6671088,38888798,10719563,36944652,36703732,9646545,29287523,24156038,24502755}},"employment":[{"organization-name":"kin-ron","start-date":date("2004-07-20"),"end-date":date("2006-03-10")}]}
+{"id":9207832,"id-copy":9207832,"alias":"Tammy","name":"TammyHozier","user-since":datetime("2005-08-24T14:34:19"),"user-since-copy":datetime("2005-08-24T14:34:19"),"friend-ids":{{26919119,35729176,28949827}},"employment":[{"organization-name":"geomedia","start-date":date("2006-09-14")}]}
+{"id":11364871,"id-copy":11364871,"alias":"Darrell","name":"DarrellTaggart","user-since":datetime("2007-02-14T07:06:21"),"user-since-copy":datetime("2007-02-14T07:06:21"),"friend-ids":{{42942141,33727432,32050372,39330410,38031970,18321427,4533038,45054607,34474798,29859123,17215101,24811589,12250229,4712867,23411515,10287620,37707941}},"employment":[{"organization-name":"Ontohothex","start-date":date("2006-11-26"),"end-date":date("2007-02-18")}]}
+{"id":11748019,"id-copy":11748019,"alias":"Malinda","name":"MalindaMoberly","user-since":datetime("2005-06-21T22:34:38"),"user-since-copy":datetime("2005-06-21T22:34:38"),"friend-ids":{{46792750,47197275,45940765,43931611,33201251,32508732,23681521,35069089,43652710,22676488,5098654,29592897,18671070,40200423}},"employment":[{"organization-name":"ganjalax","start-date":date("2004-08-18")}]}
+{"id":10444585,"id-copy":10444585,"alias":"Harrietta","name":"HarriettaDunkle","user-since":datetime("2012-01-26T16:14:19"),"user-since-copy":datetime("2012-01-26T16:14:19"),"friend-ids":{{9013750,39577621,40067238,24177261,41169182,5939218,13820152,47741655}},"employment":[{"organization-name":"Fixelectrics","start-date":date("2004-06-13")}]}
+{"id":10803184,"id-copy":10803184,"alias":"Daria","name":"DariaPyle","user-since":datetime("2010-11-22T05:29:27"),"user-since-copy":datetime("2010-11-22T05:29:27"),"friend-ids":{{26747755,39431389,24370112,37832812,20626868,30614988,38041392,31908762,47561829,45121087,24496373,32944554,16470795,11915899,29900938,4003497,38829225,36390033,36474051}},"employment":[{"organization-name":"physcane","start-date":date("2009-10-03")}]}
+{"id":11400016,"id-copy":11400016,"alias":"Beaumont","name":"BeaumontMiller","user-since":datetime("2008-05-12T07:13:22"),"user-since-copy":datetime("2008-05-12T07:13:22"),"friend-ids":{{41935126,36767417,10582797,47501456,43527117,2821865,27905409,13531461,16278289,9565333,15686197,15195167,29350985,8804024,31606110,44124513,15106563,26509959,47480296,13623445,17378703,33568332,19922072,12746355}},"employment":[{"organization-name":"whitestreet","start-date":date("2002-03-04")}]}
+{"id":10808284,"id-copy":10808284,"alias":"Natalie","name":"NatalieJewell","user-since":datetime("2007-04-15T14:17:38"),"user-since-copy":datetime("2007-04-15T14:17:38"),"friend-ids":{{20839191,18422391,2571767,39525211,38867255,13491856}},"employment":[{"organization-name":"Basecone","start-date":date("2005-09-10"),"end-date":date("2011-01-20")}]}
+{"id":9955486,"id-copy":9955486,"alias":"Jerrod","name":"JerrodBeach","user-since":datetime("2007-04-18T07:24:36"),"user-since-copy":datetime("2007-04-18T07:24:36"),"friend-ids":{{9760902,36268051,11373781,42337286,41818514,20451257,23673069,14313303,6548991,34820597,17346574,46871090,263833,38179383,14434022}},"employment":[{"organization-name":"Roundhex","start-date":date("2003-09-01"),"end-date":date("2007-06-11")}]}
+{"id":10901047,"id-copy":10901047,"alias":"Salvador","name":"SalvadorBynum","user-since":datetime("2012-01-13T02:30:17"),"user-since-copy":datetime("2012-01-13T02:30:17"),"friend-ids":{{29122263,27975257,7988516,9270552,17837898,42339445,46097101,32303800,17233223,10656090,36709955,17535336,27157992,30360627,15304415,28922979,27243261,9307382,43171015,31593421,21246902,40452339,25735551,23716187}},"employment":[{"organization-name":"Striptaxon","start-date":date("2010-11-27")}]}
+{"id":11412640,"id-copy":11412640,"alias":"Larry","name":"LarryEisaman","user-since":datetime("2005-04-23T10:38:04"),"user-since-copy":datetime("2005-04-23T10:38:04"),"friend-ids":{{15063821,35006785,18241384,5967937,45426140,44234765,3244540,3222784,36330320}},"employment":[{"organization-name":"Newhotplus","start-date":date("2001-07-05")}]}
+{"id":10079965,"id-copy":10079965,"alias":"Mason","name":"MasonReamer","user-since":datetime("2008-08-10T02:16:36"),"user-since-copy":datetime("2008-08-10T02:16:36"),"friend-ids":{{37149190,37736572,35955709,28586597,45460389}},"employment":[{"organization-name":"Salthex","start-date":date("2002-01-18"),"end-date":date("2010-12-09")}]}
+{"id":11617963,"id-copy":11617963,"alias":"Sherry","name":"SherryPirl","user-since":datetime("2010-08-26T06:37:30"),"user-since-copy":datetime("2010-08-26T06:37:30"),"friend-ids":{{30179664,7140787,14622079,5810238,32189583,17103583}},"employment":[{"organization-name":"Statcode","start-date":date("2000-02-07"),"end-date":date("2004-11-24")}]}
+{"id":10073440,"id-copy":10073440,"alias":"Mat","name":"MatHasely","user-since":datetime("2007-02-15T12:28:32"),"user-since-copy":datetime("2007-02-15T12:28:32"),"friend-ids":{{18317132,16303558,35197704,41199497,17394418,18594954,13332602,15164806,20807780,18284264,17164369,6418744,26535302,47287046,7169299,22825706,34007482,38108004,14449725,16993574,28055503}},"employment":[{"organization-name":"Zuncan","start-date":date("2005-09-04"),"end-date":date("2006-06-02")}]}
+{"id":10391044,"id-copy":10391044,"alias":"Kendrick","name":"KendrickNabholz","user-since":datetime("2007-10-11T19:49:13"),"user-since-copy":datetime("2007-10-11T19:49:13"),"friend-ids":{{39264696,35794708,222108,29542536,34470710,16736694,36282306,12411530,12507843,30193842,45764599,32250152,16472135,26507230,17443301,16787960,17651924,37659951,28610616,12928071}},"employment":[{"organization-name":"Zununoing","start-date":date("2007-05-07")}]}
+{"id":11016238,"id-copy":11016238,"alias":"Justy","name":"JustyShaner","user-since":datetime("2008-06-17T22:08:29"),"user-since-copy":datetime("2008-06-17T22:08:29"),"friend-ids":{{23689951,17071721,9194411,34128749,46316500,31173605,32802286,26107462,6561314,9993897,14746369,7297148,41466258}},"employment":[{"organization-name":"Freshfix","start-date":date("2003-12-12"),"end-date":date("2007-04-12")}]}
+{"id":11302930,"id-copy":11302930,"alias":"Eustace","name":"EustaceKava","user-since":datetime("2011-08-24T18:08:32"),"user-since-copy":datetime("2011-08-24T18:08:32"),"friend-ids":{{31173988,7044500,11649679,34385410,3097267,24759223,20452579,7436501,4500062,765860,14592959,582267,25586360,6035361,38333776,47384154,22158173}},"employment":[{"organization-name":"Trustbam","start-date":date("2004-05-24")}]}
+{"id":9361930,"id-copy":9361930,"alias":"Leonard","name":"LeonardAshbaugh","user-since":datetime("2008-06-13T07:49:33"),"user-since-copy":datetime("2008-06-13T07:49:33"),"friend-ids":{{33929562,22722370,18562061,44346144,38834006,1660309,17690686,8299074,13219630,42802095,2203402,47180979,43715995,24339545,42132653,32010945,18200992,5115504}},"employment":[{"organization-name":"Xx-technology","start-date":date("2012-04-15")}]}
+{"id":10270597,"id-copy":10270597,"alias":"Ava","name":"AvaTanner","user-since":datetime("2010-04-23T11:49:39"),"user-since-copy":datetime("2010-04-23T11:49:39"),"friend-ids":{{38894360,9403074,25855965,36511208,4947767,10318201,3532083,28684767,22730535,17994309,21209113,14980333,5611975,31951870,16697364,5033131,13637894,18107216,9769275,25479923,15320268,28897820,22865104}},"employment":[{"organization-name":"Redelectronics","start-date":date("2011-04-12"),"end-date":date("2011-09-07")}]}
+{"id":10116496,"id-copy":10116496,"alias":"Gena","name":"GenaJerome","user-since":datetime("2005-03-04T21:38:41"),"user-since-copy":datetime("2005-03-04T21:38:41"),"friend-ids":{{11698908,11838778,10546816,13504928,25681727,20198355,28316946,13835662,16328293,39540292,43990464,31393679,34806990,19167324,8558031,37794176,14389975}},"employment":[{"organization-name":"Plexlane","start-date":date("2003-10-01"),"end-date":date("2006-06-13")}]}
+{"id":9356098,"id-copy":9356098,"alias":"Juliana","name":"JulianaAnderson","user-since":datetime("2007-04-26T20:13:07"),"user-since-copy":datetime("2007-04-26T20:13:07"),"friend-ids":{{3850702,46858392,20177889,34485932,20958453,26839176,23562562,47962945,43961803,19857248,29816714,14695228,35327929,16196977,11908428,30035277,23919929}},"employment":[{"organization-name":"whitestreet","start-date":date("2009-06-04"),"end-date":date("2009-05-05")}]}
+{"id":11693350,"id-copy":11693350,"alias":"Crystal","name":"CrystalDickinson","user-since":datetime("2007-02-08T08:05:12"),"user-since-copy":datetime("2007-02-08T08:05:12"),"friend-ids":{{32246301,35277320,38987334,3391139,30437594,35314588,32659406,19055708,5245289,1155014,9266846,20085529,27878886,25128707,46223557,16459237,41315912,26681594}},"employment":[{"organization-name":"Strongtone","start-date":date("2011-07-03"),"end-date":date("2011-08-05")}]}
+{"id":10766221,"id-copy":10766221,"alias":"Rosalyn","name":"RosalynBaxter","user-since":datetime("2009-04-16T15:46:54"),"user-since-copy":datetime("2009-04-16T15:46:54"),"friend-ids":{{43759575,1264811,9906031,21579594,45786210,14876191,10711745,25134652,25426644,29987806,1953812,29568099,38860088,7073296,13746927,11395655,36208297,25317651,21356968}},"employment":[{"organization-name":"tresline","start-date":date("2000-07-04")}]}
+{"id":9866572,"id-copy":9866572,"alias":"Evelina","name":"EvelinaBerry","user-since":datetime("2006-12-16T03:56:00"),"user-since-copy":datetime("2006-12-16T03:56:00"),"friend-ids":{{13883615,43198063,30615747,3228427,23840450,43443245,17107485,34691909,44890462,47992198,46475465,28790498,7693182,41338502,6694688,17592193,9966336,40899188,16363000,43996364}},"employment":[{"organization-name":"Keytech","start-date":date("2004-03-01"),"end-date":date("2008-08-21")}]}
+{"id":9205834,"id-copy":9205834,"alias":"Tristin","name":"TristinWalker","user-since":datetime("2012-04-25T01:08:05"),"user-since-copy":datetime("2012-04-25T01:08:05"),"friend-ids":{{2222398,15073251,16222879,24405969,32651599,44500557,31699173,41724026,1745441,9674348,29594086,26580583,42258300,36027050,3204087,2147469,36519580}},"employment":[{"organization-name":"highfax","start-date":date("2007-02-09")}]}
+{"id":9521683,"id-copy":9521683,"alias":"Tennille","name":"TennilleHamilton","user-since":datetime("2009-04-21T20:56:25"),"user-since-copy":datetime("2009-04-21T20:56:25"),"friend-ids":{{32048407,3619952,41652292,45570368,31678290,11241324}},"employment":[{"organization-name":"itlab","start-date":date("2010-10-15")}]}
+{"id":9329746,"id-copy":9329746,"alias":"Albert","name":"AlbertZundel","user-since":datetime("2005-11-01T23:41:02"),"user-since-copy":datetime("2005-11-01T23:41:02"),"friend-ids":{{44252308,14483702,27233282,24263669,35409140,38591765,42901786,24502313,6384822,36359249,36816246,16578182,530819,29481837,12698700,6101521,11990316,35327955,10435272}},"employment":[{"organization-name":"Doncare","start-date":date("2003-08-06"),"end-date":date("2010-09-22")}]}
+{"id":11811079,"id-copy":11811079,"alias":"Kenelm","name":"KenelmKellogg","user-since":datetime("2006-05-14T04:13:36"),"user-since-copy":datetime("2006-05-14T04:13:36"),"friend-ids":{{28287762,45591894,12026636,34381293,17018521,37239852,5735876,8145944,34171842,32986088,16537938,20530369,35161854,1076550,26081966,35666231}},"employment":[{"organization-name":"Xx-drill","start-date":date("2001-02-03")}]}
+{"id":10513507,"id-copy":10513507,"alias":"Jasmin","name":"JasminHatfield","user-since":datetime("2009-06-25T22:45:16"),"user-since-copy":datetime("2009-06-25T22:45:16"),"friend-ids":{{31323261}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2000-05-12"),"end-date":date("2003-05-07")}]}
+{"id":10412287,"id-copy":10412287,"alias":"Wren","name":"WrenElizabeth","user-since":datetime("2009-06-25T07:26:48"),"user-since-copy":datetime("2009-06-25T07:26:48"),"friend-ids":{{23487913,35496582,14824955,5998721,10925419,38937432,6285652}},"employment":[{"organization-name":"Sublamdox","start-date":date("2012-07-20"),"end-date":date("2012-07-12")}]}
+{"id":10751260,"id-copy":10751260,"alias":"Chrysanta","name":"ChrysantaSanforth","user-since":datetime("2009-06-02T12:54:32"),"user-since-copy":datetime("2009-06-02T12:54:32"),"friend-ids":{{6064707,44017707,22957433,38426343,24694205,1061085,24827089,12192854,40718843}},"employment":[{"organization-name":"Zununoing","start-date":date("2011-01-19"),"end-date":date("2011-10-02")}]}
+{"id":11190361,"id-copy":11190361,"alias":"Jancis","name":"JancisFeufer","user-since":datetime("2005-08-04T13:00:03"),"user-since-copy":datetime("2005-08-04T13:00:03"),"friend-ids":{{29421411,15938833,13248806,1321174,32401361,34058563,39735399,35531531,2631116,1167996,18366452,45021961,246133}},"employment":[{"organization-name":"Quadlane","start-date":date("2003-08-27")}]}
+{"id":11273239,"id-copy":11273239,"alias":"Alanis","name":"AlanisNeely","user-since":datetime("2009-04-11T16:49:56"),"user-since-copy":datetime("2009-04-11T16:49:56"),"friend-ids":{{16788046,3222185,46272663,16782006,29597609,9709951,37694695,39662749,18430270,38598018,40033174,34984089,8435528,2669100,18469173,25201258,29975180,16379939,24603,2573554,16344157,16880724,2437581}},"employment":[{"organization-name":"Ontotanin","start-date":date("2001-10-01"),"end-date":date("2006-08-24")}]}
+{"id":10967254,"id-copy":10967254,"alias":"Andre","name":"AndreCowper","user-since":datetime("2011-12-21T20:22:47"),"user-since-copy":datetime("2011-12-21T20:22:47"),"friend-ids":{{23645341,16267661,7660549,24716202,20945538,10125828,1712260,5309070,16802418,18273953,12670834}},"employment":[{"organization-name":"Vivaace","start-date":date("2003-12-25"),"end-date":date("2004-04-09")}]}
+{"id":11633284,"id-copy":11633284,"alias":"Quinn","name":"QuinnMillhouse","user-since":datetime("2006-08-06T07:42:49"),"user-since-copy":datetime("2006-08-06T07:42:49"),"friend-ids":{{15791690,46827169,41678324,25101779,24496106,29442447,29240215,23819212,11076551,27248100,1506119,37415860}},"employment":[{"organization-name":"Greencare","start-date":date("2008-01-06")}]}
+{"id":10224400,"id-copy":10224400,"alias":"Malvina","name":"MalvinaPery","user-since":datetime("2009-01-25T03:41:22"),"user-since-copy":datetime("2009-01-25T03:41:22"),"friend-ids":{{17095877,17062955,13129292,31635980,32747924,902714,32032985,44944935,30544897,44429244}},"employment":[{"organization-name":"Voltlane","start-date":date("2001-01-11"),"end-date":date("2011-04-10")}]}
+{"id":9826402,"id-copy":9826402,"alias":"Rachyl","name":"RachylRumbaugh","user-since":datetime("2006-01-05T03:38:59"),"user-since-copy":datetime("2006-01-05T03:38:59"),"friend-ids":{{11891915,15900581,38420311,21084667,24569500,9181299,32167823,9967774,18138704,10742133,29173609,1113683,21048344,33794587,42308958,9303744}},"employment":[{"organization-name":"Goldcity","start-date":date("2005-04-24"),"end-date":date("2008-08-17")}]}
+{"id":10269349,"id-copy":10269349,"alias":"Oneida","name":"OneidaJube","user-since":datetime("2010-11-18T02:17:28"),"user-since-copy":datetime("2010-11-18T02:17:28"),"friend-ids":{{12058841,5816839,33989309,42710608,27128355,22765769,30666197,9009086,7254731,41783149,10080163,38431373,35086196,3607650}},"employment":[{"organization-name":"Trustbam","start-date":date("2000-12-10")}]}
+{"id":11725939,"id-copy":11725939,"alias":"Clover","name":"CloverAlice","user-since":datetime("2007-07-12T05:17:52"),"user-since-copy":datetime("2007-07-12T05:17:52"),"friend-ids":{{24426905,6647137,25463555,11443041,10549599,35925634,4053835,11813301,6976204,26680887,29934690,7935338,45092791,30510709}},"employment":[{"organization-name":"Hexsanhex","start-date":date("2000-05-04"),"end-date":date("2000-08-24")}]}
+{"id":9939937,"id-copy":9939937,"alias":"Margeret","name":"MargeretWhite","user-since":datetime("2008-10-10T22:07:17"),"user-since-copy":datetime("2008-10-10T22:07:17"),"friend-ids":{{12369844,34252449,12412010,16942281,25231122,42326296,27054531,8338820,25466132,10175756,23763550,40035149,41030740,36493305,19615682,30813330,24869907,6934392,31309446,2545800,463498,3089623,12714051,38317605}},"employment":[{"organization-name":"Streettax","start-date":date("2012-01-19")}]}
+{"id":9522265,"id-copy":9522265,"alias":"Brendon","name":"BrendonLing","user-since":datetime("2012-08-11T12:01:34"),"user-since-copy":datetime("2012-08-11T12:01:34"),"friend-ids":{{32770998,43037450,13481444,36411834,21704194}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2012-08-30")}]}
+{"id":11894854,"id-copy":11894854,"alias":"Connor","name":"ConnorWilliamson","user-since":datetime("2011-09-16T22:24:17"),"user-since-copy":datetime("2011-09-16T22:24:17"),"friend-ids":{{19318451,47946991,1913830,45324890,47189256,39211392,6998884,4344587,24720830,4355756,19102058,34241496,39408673,1360498,7695088,25754984,21796436}},"employment":[{"organization-name":"Inchdox","start-date":date("2007-09-19"),"end-date":date("2010-07-22")}]}
+{"id":10835521,"id-copy":10835521,"alias":"Margeret","name":"MargeretEve","user-since":datetime("2010-02-13T16:16:55"),"user-since-copy":datetime("2010-02-13T16:16:55"),"friend-ids":{{40363275,44184724,42855751,10492711,561147,45516609,38567828,9695088,40235757}},"employment":[{"organization-name":"Kanelectrics","start-date":date("2012-06-08"),"end-date":date("2012-06-27")}]}
+{"id":11912419,"id-copy":11912419,"alias":"Wallis","name":"WallisFuchs","user-since":datetime("2012-01-07T08:13:18"),"user-since-copy":datetime("2012-01-07T08:13:18"),"friend-ids":{{11115387,19639311,33957302,8746808,20140328,35866755,29492622,24246926,14412186,1610423,1139443,23667812,6972455,18354247,7072427,43742595,20711654,7179925,66544,12066267,8914321,35602734}},"employment":[{"organization-name":"Fixdintex","start-date":date("2008-10-23"),"end-date":date("2008-06-18")}]}
+{"id":9709663,"id-copy":9709663,"alias":"Trevor","name":"TrevorSell","user-since":datetime("2008-08-28T18:18:54"),"user-since-copy":datetime("2008-08-28T18:18:54"),"friend-ids":{{13788189,27667188,588943,1574745,5763893,19661124,45630528,47078471,42976078,32943975}},"employment":[{"organization-name":"Villa-dox","start-date":date("2007-07-04")}]}
+{"id":10853926,"id-copy":10853926,"alias":"Kennard","name":"KennardGarland","user-since":datetime("2007-11-28T20:40:40"),"user-since-copy":datetime("2007-11-28T20:40:40"),"friend-ids":{{47687855,28575858}},"employment":[{"organization-name":"Newcom","start-date":date("2005-07-17")}]}
+{"id":11130781,"id-copy":11130781,"alias":"Kenia","name":"KeniaMiller","user-since":datetime("2008-05-27T02:28:18"),"user-since-copy":datetime("2008-05-27T02:28:18"),"friend-ids":{{43139868,16103105,25352928,23612973,9645914,20517323,40438742,47972276,7395189,44164898,2805123,33235701,39846510,21170026,14223369,14077979}},"employment":[{"organization-name":"Tripplelane","start-date":date("2011-06-24"),"end-date":date("2011-04-08")}]}
+{"id":9408688,"id-copy":9408688,"alias":"Goddard","name":"GoddardWeisgarber","user-since":datetime("2011-05-21T13:18:54"),"user-since-copy":datetime("2011-05-21T13:18:54"),"friend-ids":{{2820008,31637633,35026624,544628,2552858}},"employment":[{"organization-name":"jaydax","start-date":date("2007-11-26")}]}
+{"id":10346338,"id-copy":10346338,"alias":"Caelie","name":"CaelieYates","user-since":datetime("2011-11-10T19:17:38"),"user-since-copy":datetime("2011-11-10T19:17:38"),"friend-ids":{{3910270,7940512,32351319,27966615,33829964,34529061,19420019,7423616,22246488,7284253,8419860,43330144}},"employment":[{"organization-name":"Kanelectrics","start-date":date("2005-02-07"),"end-date":date("2011-09-05")}]}
+{"id":10686646,"id-copy":10686646,"alias":"Deborah","name":"DeborahRosenstiehl","user-since":datetime("2012-06-18T16:51:32"),"user-since-copy":datetime("2012-06-18T16:51:32"),"friend-ids":{{34005621,6910583,11226890,1333457,13615971,15332838,30484423,38261521,39526604,12093262,15397660,29644860,36715060,16753181}},"employment":[{"organization-name":"U-electrics","start-date":date("2005-07-01"),"end-date":date("2007-10-22")}]}
+{"id":10071475,"id-copy":10071475,"alias":"Kyra","name":"KyraWile","user-since":datetime("2010-08-21T20:27:23"),"user-since-copy":datetime("2010-08-21T20:27:23"),"friend-ids":{{24326501,3159228,33973593,47221189,17474184,17812891}},"employment":[{"organization-name":"Unijobam","start-date":date("2010-04-14")}]}
+{"id":9426544,"id-copy":9426544,"alias":"Joshawa","name":"JoshawaHiles","user-since":datetime("2012-04-28T09:48:20"),"user-since-copy":datetime("2012-04-28T09:48:20"),"friend-ids":{{16780903}},"employment":[{"organization-name":"Keytech","start-date":date("2002-07-01")}]}
+{"id":10972447,"id-copy":10972447,"alias":"Loretta","name":"LorettaBriggs","user-since":datetime("2005-07-01T10:25:33"),"user-since-copy":datetime("2005-07-01T10:25:33"),"friend-ids":{{6898813,6606991,14092255,9865734,23960698,47354873,19345256}},"employment":[{"organization-name":"Sanjodax","start-date":date("2005-06-02")}]}
+{"id":10867444,"id-copy":10867444,"alias":"Tetty","name":"TettyZundel","user-since":datetime("2012-07-26T17:54:45"),"user-since-copy":datetime("2012-07-26T17:54:45"),"friend-ids":{{17830961,13154371,12005619,15279158,15766172,3071670,4314512,29378453,33264674,32657723,37875054,6208013,23310809,11994927,9787690,25069760,11104605,44517542,45829337,26593992}},"employment":[{"organization-name":"Doublezone","start-date":date("2001-09-25")}]}
+{"id":9434542,"id-copy":9434542,"alias":"Alice","name":"AliceRopes","user-since":datetime("2011-09-10T10:32:17"),"user-since-copy":datetime("2011-09-10T10:32:17"),"friend-ids":{{30233815,23593045,243865,46494768,15852416,2627657,12253908,11415849,36381160,25773586,9952015,20363967,45499740,15573031,2939342,24137982,34026341,34111551,30963526,7116453}},"employment":[{"organization-name":"Qvohouse","start-date":date("2008-07-09")}]}
+{"id":11216260,"id-copy":11216260,"alias":"Randy","name":"RandyEckhardstein","user-since":datetime("2006-12-05T07:09:34"),"user-since-copy":datetime("2006-12-05T07:09:34"),"friend-ids":{{39744737,14315897,1342674,1761832,41393930,21351330,17845632,39034426,15297881,11656496,11376855}},"employment":[{"organization-name":"Unijobam","start-date":date("2009-12-19")}]}
+{"id":9187549,"id-copy":9187549,"alias":"Lenny","name":"LennyField","user-since":datetime("2008-09-11T10:50:10"),"user-since-copy":datetime("2008-09-11T10:50:10"),"friend-ids":{{26505249,4392946,32062169,45628101,22865593,4982483,13425537,18846467,36122039,2998293,19787439,22246499,43133873,30573462,36272473,41691126,43929640,43759980,25546305}},"employment":[{"organization-name":"Groovetex","start-date":date("2004-08-13"),"end-date":date("2010-03-08")}]}
+{"id":11681410,"id-copy":11681410,"alias":"Wendell","name":"WendellGarneys","user-since":datetime("2007-07-23T13:10:29"),"user-since-copy":datetime("2007-07-23T13:10:29"),"friend-ids":{{11124106,3438927,28547601,18074764,35037765,25438231,8196141,26000844,6063826,22981069,31549929,33158093,40748728,12245244,2442169,7879517,877005,24286984}},"employment":[{"organization-name":"Freshfix","start-date":date("2008-02-10"),"end-date":date("2008-05-15")}]}
+{"id":10533343,"id-copy":10533343,"alias":"Gwendolen","name":"GwendolenHanseu","user-since":datetime("2007-02-04T19:56:51"),"user-since-copy":datetime("2007-02-04T19:56:51"),"friend-ids":{{25281794,21814505,11684475,5599252,17261378,11061422,27392332,47872606,39198697,17314413,4034634,42776559,43885593,24835625,18150148,4946129,9288372,5675162,34976580}},"employment":[{"organization-name":"Fax-fax","start-date":date("2010-07-10")}]}
+{"id":10514428,"id-copy":10514428,"alias":"Eliseo","name":"EliseoHoffhants","user-since":datetime("2012-08-24T08:40:51"),"user-since-copy":datetime("2012-08-24T08:40:51"),"friend-ids":{{45751891,26026786,24531389,26239368,34021241}},"employment":[{"organization-name":"Newhotplus","start-date":date("2010-03-01"),"end-date":date("2010-08-02")}]}
+{"id":10451932,"id-copy":10451932,"alias":"Kory","name":"KoryRomanoff","user-since":datetime("2008-09-27T13:29:18"),"user-since-copy":datetime("2008-09-27T13:29:18"),"friend-ids":{{21328124,47569968,22569123,34316877,36016117,19944396,34862141,14875173,3888684,25235679,7930355,24991146,2862320,9552488,23394143,6292732,23109993}},"employment":[{"organization-name":"Codetechno","start-date":date("2008-04-25"),"end-date":date("2010-03-18")}]}
+{"id":11435779,"id-copy":11435779,"alias":"Jonty","name":"JontyLarson","user-since":datetime("2012-04-11T08:34:47"),"user-since-copy":datetime("2012-04-11T08:34:47"),"friend-ids":{{37343432,9979565,14647518,32490112,26673699,22447290,40923710,47426439}},"employment":[{"organization-name":"Doncare","start-date":date("2010-08-24"),"end-date":date("2011-06-21")}]}
+{"id":9577867,"id-copy":9577867,"alias":"Lavette","name":"LavetteSnyder","user-since":datetime("2007-02-22T10:01:04"),"user-since-copy":datetime("2007-02-22T10:01:04"),"friend-ids":{{25749553,31379974,15118772,38725424,26760226,8908746,20299291,20288328,19659485,22400738,477700,20253845,12753420,46016251,29518581,21898853,19015599,3455762,19350275,2630122}},"employment":[{"organization-name":"Solophase","start-date":date("2011-04-22")}]}
+{"id":10878553,"id-copy":10878553,"alias":"Fido","name":"FidoWillcox","user-since":datetime("2007-01-10T01:06:54"),"user-since-copy":datetime("2007-01-10T01:06:54"),"friend-ids":{{28379360,45087756,15173549,15693878,23925453,44178250,26895550,35260808,9946110}},"employment":[{"organization-name":"Dandamace","start-date":date("2012-02-09"),"end-date":date("2012-06-24")}]}
+{"id":10338907,"id-copy":10338907,"alias":"Leah","name":"LeahStroble","user-since":datetime("2010-12-07T08:23:00"),"user-since-copy":datetime("2010-12-07T08:23:00"),"friend-ids":{{25263375,47112518,47910837,14446727,35708710,41365949,8534511,34992353,1706302,21380997,47197876,29441929,4157771,8674755,14520863,22041433,47176591,4072306,47354501}},"employment":[{"organization-name":"Sancone","start-date":date("2003-03-25")}]}
+{"id":9945166,"id-copy":9945166,"alias":"Lilly","name":"LillyPirl","user-since":datetime("2009-10-26T11:59:59"),"user-since-copy":datetime("2009-10-26T11:59:59"),"friend-ids":{{44569094,5885974,43165146,40353390,45117914,35995608,22535699,46288114,47171829,14193764,45832182,4957844,2623547,37294528}},"employment":[{"organization-name":"freshdox","start-date":date("2009-12-15"),"end-date":date("2011-11-20")}]}
+{"id":10300027,"id-copy":10300027,"alias":"Cassie","name":"CassieCarmichael","user-since":datetime("2007-02-17T16:12:21"),"user-since-copy":datetime("2007-02-17T16:12:21"),"friend-ids":{{18690821,9246387,5425670,8058755,32156367,29092478}},"employment":[{"organization-name":"Xx-drill","start-date":date("2000-03-06")}]}
+{"id":11221033,"id-copy":11221033,"alias":"Vernon","name":"VernonLear","user-since":datetime("2006-04-19T13:02:26"),"user-since-copy":datetime("2006-04-19T13:02:26"),"friend-ids":{{45628776,31762296,22963223,10079920,20931037,41768759,25910794,41882156,36691498,1652094,25804751,35757270,40057670,37961622,7430384,1498630,7636920,17109852,12569850,47366298,22902730,5889994,21003934,1929823}},"employment":[{"organization-name":"Codetechno","start-date":date("2000-04-18")}]}
+{"id":11978782,"id-copy":11978782,"alias":"Louiza","name":"LouizaLlora","user-since":datetime("2012-06-24T06:19:05"),"user-since-copy":datetime("2012-06-24T06:19:05"),"friend-ids":{{36495107,35125435,30347420,17703387,40909002}},"employment":[{"organization-name":"Indiex","start-date":date("2008-05-25")}]}
+{"id":11529730,"id-copy":11529730,"alias":"Linwood","name":"LinwoodZadovsky","user-since":datetime("2007-03-13T03:41:20"),"user-since-copy":datetime("2007-03-13T03:41:20"),"friend-ids":{{23516069,24312236,23750591,36982495,36483830}},"employment":[{"organization-name":"subtam","start-date":date("2008-01-25")}]}
+{"id":9477040,"id-copy":9477040,"alias":"Chery","name":"CheryWatson","user-since":datetime("2012-05-02T14:27:00"),"user-since-copy":datetime("2012-05-02T14:27:00"),"friend-ids":{{36360097,36835617,25761112,30806900,22340413,16802957}},"employment":[{"organization-name":"kin-ron","start-date":date("2008-12-26"),"end-date":date("2009-03-17")}]}
+{"id":10742182,"id-copy":10742182,"alias":"Tel","name":"TelBowchiew","user-since":datetime("2009-09-23T02:51:14"),"user-since-copy":datetime("2009-09-23T02:51:14"),"friend-ids":{{17515416,42010238,23580669,26008148,35744494}},"employment":[{"organization-name":"Villa-dox","start-date":date("2006-10-05"),"end-date":date("2007-05-26")}]}
+{"id":10851133,"id-copy":10851133,"alias":"Wilbur","name":"WilburDiegel","user-since":datetime("2005-08-20T01:37:10"),"user-since-copy":datetime("2005-08-20T01:37:10"),"friend-ids":{{44811869,15362002,5320359,4756538,40097009,905334,44595717,3685695,35645656,2090160,35124514,21715286,26713020,5816017,15598653,6425314,10423130,29593106,14054734,1780417,38517315,25570577,5038946}},"employment":[{"organization-name":"Sanjodax","start-date":date("2004-05-04")}]}
+{"id":10777072,"id-copy":10777072,"alias":"Fairy","name":"FairyAgg","user-since":datetime("2011-08-22T17:08:52"),"user-since-copy":datetime("2011-08-22T17:08:52"),"friend-ids":{{30447177,24535470,1763903,4456057,35013322}},"employment":[{"organization-name":"silfind","start-date":date("2009-02-19")}]}
+{"id":11988241,"id-copy":11988241,"alias":"Cyrilla","name":"CyrillaRohtin","user-since":datetime("2005-02-10T08:24:14"),"user-since-copy":datetime("2005-02-10T08:24:14"),"friend-ids":{{32725541,26677413,29278988,218049,19833496,20655804,27991386,5326490,28583388,41013948,35541276,41552165,8526660}},"employment":[{"organization-name":"Scotcity","start-date":date("2004-07-20"),"end-date":date("2004-08-19")}]}
+{"id":9482569,"id-copy":9482569,"alias":"Marty","name":"MartyBurnett","user-since":datetime("2006-03-21T10:10:40"),"user-since-copy":datetime("2006-03-21T10:10:40"),"friend-ids":{{5791578,3884688,7686005}},"employment":[{"organization-name":"Solfix","start-date":date("2009-02-01")}]}
+{"id":9259234,"id-copy":9259234,"alias":"Abigail","name":"AbigailNicola","user-since":datetime("2009-08-11T09:18:47"),"user-since-copy":datetime("2009-08-11T09:18:47"),"friend-ids":{{5465164,47505082}},"employment":[{"organization-name":"Newphase","start-date":date("2006-02-22"),"end-date":date("2007-10-02")}]}
+{"id":9560251,"id-copy":9560251,"alias":"Nivek","name":"NivekJowers","user-since":datetime("2007-02-04T08:02:07"),"user-since-copy":datetime("2007-02-04T08:02:07"),"friend-ids":{{15730417,36745553,26133088,38675683,14617495,39244216,4651791,639869,8377526,15158817,13368295,15386494,5649384,8449938,34497809,6644713,45481442,27678941,14214532,5753112,9991855,25975202,9530884,19069924}},"employment":[{"organization-name":"Kongreen","start-date":date("2003-08-15")}]}
+{"id":11619817,"id-copy":11619817,"alias":"Conor","name":"ConorIsaman","user-since":datetime("2007-07-19T03:08:58"),"user-since-copy":datetime("2007-07-19T03:08:58"),"friend-ids":{{3118516,11993690,44936801,20826732,45978958,5214526,29651996,39212065,47935248,13306157,33084407,537249,42089040,7553609,42024531,23482433,45497814,26865252,42135224,41353574,28567135,7898064}},"employment":[{"organization-name":"Ronholdings","start-date":date("2002-04-26")}]}
+{"id":10391179,"id-copy":10391179,"alias":"Raymond","name":"RaymondHoopengarner","user-since":datetime("2006-04-06T18:32:20"),"user-since-copy":datetime("2006-04-06T18:32:20"),"friend-ids":{{35664656,36940003,35836359,25322876,45895708,14553421}},"employment":[{"organization-name":"Greencare","start-date":date("2005-08-05"),"end-date":date("2007-01-09")}]}
+{"id":11534575,"id-copy":11534575,"alias":"Sena","name":"SenaWeidemann","user-since":datetime("2008-05-25T01:11:53"),"user-since-copy":datetime("2008-05-25T01:11:53"),"friend-ids":{{8564372,20258364,35812476,36877724,30983504,17757915,42833517}},"employment":[{"organization-name":"Zununoing","start-date":date("2003-04-22")}]}
+{"id":10069987,"id-copy":10069987,"alias":"Andrina","name":"AndrinaFisher","user-since":datetime("2012-07-21T07:28:30"),"user-since-copy":datetime("2012-07-21T07:28:30"),"friend-ids":{{42024943,39627436,28414443,36703363,45477433,37499278,28548620,6687009,22700392,47812034,16805789,33222895,36328879,20191886,32457353,14008353}},"employment":[{"organization-name":"Hot-tech","start-date":date("2004-12-11"),"end-date":date("2004-09-07")}]}
+{"id":9575338,"id-copy":9575338,"alias":"Isabell","name":"IsabellWain","user-since":datetime("2011-07-05T12:26:43"),"user-since-copy":datetime("2011-07-05T12:26:43"),"friend-ids":{{42651024,15652966,27390748,19369775,44130969,45269514,210916,36228917,31857984,11676544,42752689,14021599,31749945,9405328,37567152,17083209,32654328,39607403,18699149,37082017,6059914,881724}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2003-06-04")}]}
+{"id":9602242,"id-copy":9602242,"alias":"Marc","name":"MarcDimsdale","user-since":datetime("2005-10-03T23:32:18"),"user-since-copy":datetime("2005-10-03T23:32:18"),"friend-ids":{{34004502,24469994,2140538,1486939,6895407,13348535,22384921,11662871,21398307,33070732,45602509,26146770,24148813,45988030,22184030,855669,36390708,30883354,26360628,29836897,28696575}},"employment":[{"organization-name":"Tranzap","start-date":date("2004-05-15"),"end-date":date("2008-01-19")}]}
+{"id":9477919,"id-copy":9477919,"alias":"Lilly","name":"LillyLinton","user-since":datetime("2005-01-09T12:24:01"),"user-since-copy":datetime("2005-01-09T12:24:01"),"friend-ids":{{19117935,45208482,36019625,39146688,15911832}},"employment":[{"organization-name":"Icerunin","start-date":date("2011-03-03"),"end-date":date("2011-10-03")}]}
+{"id":10767553,"id-copy":10767553,"alias":"Titty","name":"TittyCross","user-since":datetime("2009-02-08T11:38:56"),"user-since-copy":datetime("2009-02-08T11:38:56"),"friend-ids":{{10869392,39422025,23051606,43241994,6257807,37258783,26946341,33120713,6481181,13410766,34576024,42401239,28793792,37331232,5979767}},"employment":[{"organization-name":"Villa-dox","start-date":date("2000-12-26"),"end-date":date("2006-01-17")}]}
+{"id":10504084,"id-copy":10504084,"alias":"Etsuko","name":"EtsukoDealtry","user-since":datetime("2012-05-11T00:35:22"),"user-since-copy":datetime("2012-05-11T00:35:22"),"friend-ids":{{27578969,40308832,15379566,8664135,21276773,43659426,28027401,23264043,23981731,19124540,36281456,38766688,37886842,20522702,28559857,9838362,30409517,14237008,41013610,41586760,37285778,29427060,45678692,32255048}},"employment":[{"organization-name":"Indiex","start-date":date("2011-10-12"),"end-date":date("2011-12-04")}]}
+{"id":10865788,"id-copy":10865788,"alias":"Ebba","name":"EbbaSwartzbaugh","user-since":datetime("2007-08-18T11:38:20"),"user-since-copy":datetime("2007-08-18T11:38:20"),"friend-ids":{{12850265,19824056,2754383,43333892,9287993,14972999,3729396,20735424}},"employment":[{"organization-name":"Doublezone","start-date":date("2001-10-07"),"end-date":date("2004-07-17")}]}
+{"id":9243769,"id-copy":9243769,"alias":"Florentino","name":"FlorentinoRiggle","user-since":datetime("2012-04-04T17:10:31"),"user-since-copy":datetime("2012-04-04T17:10:31"),"friend-ids":{{41929020,22354873}},"employment":[{"organization-name":"Ontotanin","start-date":date("2007-07-23")}]}
+{"id":10710526,"id-copy":10710526,"alias":"Heike","name":"HeikeReed","user-since":datetime("2009-08-15T19:20:30"),"user-since-copy":datetime("2009-08-15T19:20:30"),"friend-ids":{{36253853,35694929,43324582,24829816}},"employment":[{"organization-name":"silfind","start-date":date("2011-02-12"),"end-date":date("2011-01-22")}]}
+{"id":9690049,"id-copy":9690049,"alias":"Ahmed","name":"AhmedVinsant","user-since":datetime("2009-12-24T23:10:10"),"user-since-copy":datetime("2009-12-24T23:10:10"),"friend-ids":{{9425379,24773026,47645199,12718095,32145472,30931581,11512330,46898742,26190870,38985851,40692118,34327720,47432207}},"employment":[{"organization-name":"Vivaace","start-date":date("2002-05-26")}]}
+{"id":10278550,"id-copy":10278550,"alias":"Parker","name":"ParkerWinton","user-since":datetime("2008-03-02T18:54:35"),"user-since-copy":datetime("2008-03-02T18:54:35"),"friend-ids":{{281420,13481584,25554653,2922131,15313837,33567564,20182917,20143660,35884326,22038516,183180}},"employment":[{"organization-name":"Villa-dox","start-date":date("2002-12-16"),"end-date":date("2010-08-04")}]}
+{"id":10305280,"id-copy":10305280,"alias":"Isabella","name":"IsabellaWilo","user-since":datetime("2007-01-03T11:54:28"),"user-since-copy":datetime("2007-01-03T11:54:28"),"friend-ids":{{46537100,26395353,23044918}},"employment":[{"organization-name":"freshdox","start-date":date("2005-01-06")}]}
+{"id":9170767,"id-copy":9170767,"alias":"Noble","name":"NobleByers","user-since":datetime("2012-04-19T03:21:33"),"user-since-copy":datetime("2012-04-19T03:21:33"),"friend-ids":{{17464807,11911237,31984348,14323306,21828766,24212960,3269277,24648466,30032203,15837021,12033801,3899014,6105665,4416812,33902540,9640452,3739829,14414940,36838129,7327467,35420130,24031049}},"employment":[{"organization-name":"Newfase","start-date":date("2007-08-11")}]}
+{"id":9087292,"id-copy":9087292,"alias":"Kiersten","name":"KierstenRawls","user-since":datetime("2005-03-21T08:42:24"),"user-since-copy":datetime("2005-03-21T08:42:24"),"friend-ids":{{5551555,2958358,17900476,23956783,31634897,12573318,32475113,47343698,40929064,39881831,38067700,3519291,19229024,4383684,13932328,16414275,8654888,16145374,26880764}},"employment":[{"organization-name":"over-it","start-date":date("2004-03-10")}]}
+{"id":10992421,"id-copy":10992421,"alias":"Ashleigh","name":"AshleighStroh","user-since":datetime("2009-10-20T03:03:48"),"user-since-copy":datetime("2009-10-20T03:03:48"),"friend-ids":{{34581685,36997971,29555907,34868441,31092587,9963667,60170,19708784,26201942,27806479,40464656,27628428,5144660,44794976,9937339}},"employment":[{"organization-name":"Ranhotfan","start-date":date("2001-11-04")}]}
+{"id":10968562,"id-copy":10968562,"alias":"Fox","name":"FoxBillimek","user-since":datetime("2012-03-24T07:32:17"),"user-since-copy":datetime("2012-03-24T07:32:17"),"friend-ids":{{8459327,11505750,30952882,30467951,6329439,33947538,19579432,25135787,41391398,32456626,6310287,31211659}},"employment":[{"organization-name":"Latsonity","start-date":date("2009-01-17")}]}
+{"id":9606691,"id-copy":9606691,"alias":"Reva","name":"RevaChristman","user-since":datetime("2010-03-04T11:53:00"),"user-since-copy":datetime("2010-03-04T11:53:00"),"friend-ids":{{21390421}},"employment":[{"organization-name":"Scotcity","start-date":date("2010-12-13")}]}
+{"id":9773836,"id-copy":9773836,"alias":"Harris","name":"HarrisAshmore","user-since":datetime("2005-11-09T08:38:57"),"user-since-copy":datetime("2005-11-09T08:38:57"),"friend-ids":{{8138978,18579002,42663609,12096643,38992166,36937135,19634600,2103929,37072923,25031081,13379299,11238246,23166598,19181943,45382447,8237252,30986231,29591835}},"employment":[{"organization-name":"highfax","start-date":date("2000-02-12")}]}
+{"id":9274378,"id-copy":9274378,"alias":"Callista","name":"CallistaCatleay","user-since":datetime("2012-01-11T05:02:51"),"user-since-copy":datetime("2012-01-11T05:02:51"),"friend-ids":{{35709258,45469345,7683235,10959232,44123341,35853639,11693773,39944820,47667622,42781782,4756825,23566535}},"employment":[{"organization-name":"Quadlane","start-date":date("2002-04-15"),"end-date":date("2003-04-03")}]}
+{"id":11570386,"id-copy":11570386,"alias":"Hollis","name":"HollisIseman","user-since":datetime("2009-07-11T12:26:25"),"user-since-copy":datetime("2009-07-11T12:26:25"),"friend-ids":{{28136044,6945424,35390131,12649451,38331381,30399822,47834313}},"employment":[{"organization-name":"subtam","start-date":date("2011-02-12")}]}
+{"id":11032186,"id-copy":11032186,"alias":"Tabby","name":"TabbySealis","user-since":datetime("2007-12-10T21:45:46"),"user-since-copy":datetime("2007-12-10T21:45:46"),"friend-ids":{{8190058,5089537,18167034,19113649,38817127,7644664,12427817,39615196,11451538,27188211,27425673,33084974,10726858,40696324,41487982,42282364,17084607,41647211,40268195,29075837,41802984,9719771,29747340,28103359}},"employment":[{"organization-name":"Redelectronics","start-date":date("2008-07-13"),"end-date":date("2010-12-04")}]}
+{"id":9320062,"id-copy":9320062,"alias":"Samantha","name":"SamanthaTanner","user-since":datetime("2010-06-25T14:13:49"),"user-since-copy":datetime("2010-06-25T14:13:49"),"friend-ids":{{19538026}},"employment":[{"organization-name":"physcane","start-date":date("2009-02-24")}]}
+{"id":11571217,"id-copy":11571217,"alias":"Modesto","name":"ModestoPark","user-since":datetime("2006-01-18T06:28:01"),"user-since-copy":datetime("2006-01-18T06:28:01"),"friend-ids":{{3765450,13287809,17696557,32161653,46823306,2818286,38794110,24894266,33129431,26474332,9356762,38679272,40502952,34470547,30005230,32074010,38611550}},"employment":[{"organization-name":"Latsonity","start-date":date("2001-09-01"),"end-date":date("2003-04-11")}]}
+{"id":10602166,"id-copy":10602166,"alias":"Karine","name":"KarineAdams","user-since":datetime("2006-03-03T20:36:12"),"user-since-copy":datetime("2006-03-03T20:36:12"),"friend-ids":{{4463206,23962283,34321170,10546383,39886106,37478996}},"employment":[{"organization-name":"Sanjodax","start-date":date("2000-10-28"),"end-date":date("2010-04-26")}]}
+{"id":9975778,"id-copy":9975778,"alias":"Marmaduke","name":"MarmadukeElizabeth","user-since":datetime("2012-07-18T02:21:55"),"user-since-copy":datetime("2012-07-18T02:21:55"),"friend-ids":{{17424696,34807936,8912699,40534595,36049658,31706902,7626256,16178188,36944385,47878361,8190132,34365280,13576207,42728095}},"employment":[{"organization-name":"Inchex","start-date":date("2006-08-19")}]}
+{"id":9041578,"id-copy":9041578,"alias":"Kristia","name":"KristiaWillcox","user-since":datetime("2012-01-09T10:29:02"),"user-since-copy":datetime("2012-01-09T10:29:02"),"friend-ids":{{29794000,34515675,3759231,14418948,35788028,34225561,20821065,27582458,4424723}},"employment":[{"organization-name":"Basecone","start-date":date("2005-06-04"),"end-date":date("2008-01-13")}]}
+{"id":9516652,"id-copy":9516652,"alias":"Emmanuel","name":"EmmanuelStrickland","user-since":datetime("2006-01-14T03:08:13"),"user-since-copy":datetime("2006-01-14T03:08:13"),"friend-ids":{{21213113,8011145,9382308,14949454,114459,30046906,40091327,22275481,14642211,5602065,15265189,22736575,12746303,46033445,17273286,39395247,6653955,14664612,35055957}},"employment":[{"organization-name":"jaydax","start-date":date("2011-10-15")}]}
+{"id":11867464,"id-copy":11867464,"alias":"Emmerson","name":"EmmersonMoore","user-since":datetime("2006-12-26T00:15:40"),"user-since-copy":datetime("2006-12-26T00:15:40"),"friend-ids":{{5310233,16498267,12436996,24801626,44135326,45729147,6922158,25920138,16324404,30272475,22873357,720070,9722837,29718785,5402637,287196,32557949}},"employment":[{"organization-name":"tresline","start-date":date("2007-06-16"),"end-date":date("2007-02-05")}]}
+{"id":11335972,"id-copy":11335972,"alias":"Emmett","name":"EmmettBaxter","user-since":datetime("2008-04-25T01:22:30"),"user-since-copy":datetime("2008-04-25T01:22:30"),"friend-ids":{{23133373,28796661,13045317,34201656,44749284,42654826,988887,5039257,18280226,30366668,22387991,32676638,24149069,6307083,17556069,16687473,4101198,41964241,39245728}},"employment":[{"organization-name":"Greencare","start-date":date("2004-11-22")}]}
+{"id":9043201,"id-copy":9043201,"alias":"Eliseo","name":"EliseoBagley","user-since":datetime("2007-05-17T10:44:18"),"user-since-copy":datetime("2007-05-17T10:44:18"),"friend-ids":{{41250222,28415639,40825493,11902499,39161617,16612650,39102228,46013732,42664763,20165539,40891614,2887877,27999503,5059039,9617378,16378780,21987749}},"employment":[{"organization-name":"whitestreet","start-date":date("2003-05-26")}]}
+{"id":9804196,"id-copy":9804196,"alias":"Micheal","name":"MichealEiford","user-since":datetime("2009-05-21T02:55:17"),"user-since-copy":datetime("2009-05-21T02:55:17"),"friend-ids":{{31376257,19749408,5790154,17891222,15712036,40911870,40765983,38804584,24619388,10957577,35370581,39352927,6063001,23702369,14716580,46589395,35232946}},"employment":[{"organization-name":"Sancone","start-date":date("2003-07-27")}]}
+{"id":11463820,"id-copy":11463820,"alias":"Gaye","name":"GayeWelty","user-since":datetime("2005-01-04T14:32:34"),"user-since-copy":datetime("2005-01-04T14:32:34"),"friend-ids":{{44428980,1291384,10830264,2433795,17582948,17416624,21578025,14538036,41470487,34384402,42863727,35119046,35673193,14814350,29380258,30253821,41180218,13945680,15533641,26510747}},"employment":[{"organization-name":"Hexsanhex","start-date":date("2011-03-01"),"end-date":date("2011-09-13")}]}
+{"id":10658977,"id-copy":10658977,"alias":"Danny","name":"DannyBailey","user-since":datetime("2006-12-12T12:28:17"),"user-since-copy":datetime("2006-12-12T12:28:17"),"friend-ids":{{27744791,5839976,37243832,42061553,15660549,26723434,25864049,8038100,47690286,29206337,6169296,1933137,6500848,45632949,6329147,15602171,13477556,25033716,9515038,4081408,42840830}},"employment":[{"organization-name":"U-electrics","start-date":date("2009-12-14"),"end-date":date("2009-03-11")}]}
+{"id":10000456,"id-copy":10000456,"alias":"Love","name":"LoveHawker","user-since":datetime("2011-03-01T20:42:05"),"user-since-copy":datetime("2011-03-01T20:42:05"),"friend-ids":{{33646270,5736885,35243769,35528678,43954964,44975821,1839952,24025196,1108928}},"employment":[{"organization-name":"Rungozoom","start-date":date("2010-11-23"),"end-date":date("2011-03-07")}]}
+{"id":9158293,"id-copy":9158293,"alias":"Cortney","name":"CortneyPainter","user-since":datetime("2006-03-15T09:03:09"),"user-since-copy":datetime("2006-03-15T09:03:09"),"friend-ids":{{42832801,24287760,37934712,43376751,24673433,14168792,46862345,46736573,21181723,2094484,30254710,45439521,26589024,45746175,13898656,13470143,9669892}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2011-06-13")}]}
+{"id":11570617,"id-copy":11570617,"alias":"Deshawn","name":"DeshawnBashline","user-since":datetime("2006-04-14T01:05:38"),"user-since-copy":datetime("2006-04-14T01:05:38"),"friend-ids":{{9319940,45556479,44222390,22928539,27909778,21162548,8657905,15375082,38338906,21416203,7519884,30405265,32148274,35560776,29973785,19277384,44256954,40425041,30328494,39977803,40280359,3079013,18841024,23001903}},"employment":[{"organization-name":"Doublezone","start-date":date("2009-02-14")}]}
+{"id":9769501,"id-copy":9769501,"alias":"Geffrey","name":"GeffreyBurch","user-since":datetime("2005-08-28T03:10:56"),"user-since-copy":datetime("2005-08-28T03:10:56"),"friend-ids":{{21060169,45384418,20564855,24708101,30231,29383832,9200835,822161,29674263,619991,38797966,14299510,13545166,33027152}},"employment":[{"organization-name":"Streettax","start-date":date("2004-01-03"),"end-date":date("2006-04-13")}]}
+{"id":9373819,"id-copy":9373819,"alias":"Man","name":"ManHarding","user-since":datetime("2005-03-19T02:36:47"),"user-since-copy":datetime("2005-03-19T02:36:47"),"friend-ids":{{10687886,6212430,40098775,8554409,18917793,9329327,38361031,27404932,29083756,28482636,38832020,7859160,14175144,3316105,16742847,8143105,13049385,22288103,36693926,26571195,6536981,32281681,41798492,36467563}},"employment":[{"organization-name":"Trustbam","start-date":date("2009-02-08")}]}
+{"id":9201610,"id-copy":9201610,"alias":"Elaine","name":"ElaineMcclymonds","user-since":datetime("2008-04-13T17:06:35"),"user-since-copy":datetime("2008-04-13T17:06:35"),"friend-ids":{{18934024,5114594,25593808}},"employment":[{"organization-name":"Hexviafind","start-date":date("2006-08-28")}]}
+{"id":10809730,"id-copy":10809730,"alias":"Algar","name":"AlgarZaun","user-since":datetime("2008-08-14T06:37:59"),"user-since-copy":datetime("2008-08-14T06:37:59"),"friend-ids":{{12676185,26087426,42241358,47854149,22179884,34701736,35541344,46257087,35091522,10779069}},"employment":[{"organization-name":"Quoline","start-date":date("2010-09-13")}]}
+{"id":10367503,"id-copy":10367503,"alias":"Tory","name":"ToryBender","user-since":datetime("2012-01-17T03:20:23"),"user-since-copy":datetime("2012-01-17T03:20:23"),"friend-ids":{{12035968,32370161,7506904,40525754,44978940,28927429,47139832,9164811,29534171,3789973}},"employment":[{"organization-name":"Transhigh","start-date":date("2009-02-05")}]}
+{"id":11350432,"id-copy":11350432,"alias":"Fletcher","name":"FletcherRowley","user-since":datetime("2012-01-22T12:30:57"),"user-since-copy":datetime("2012-01-22T12:30:57"),"friend-ids":{{43655299,46172971,29175610,22537183,30612976,21304031,40531272,6719806,42232806,18593968,29334159}},"employment":[{"organization-name":"highfax","start-date":date("2002-02-17"),"end-date":date("2011-03-16")}]}
+{"id":11668552,"id-copy":11668552,"alias":"Kassandra","name":"KassandraJames","user-since":datetime("2010-09-27T18:12:59"),"user-since-copy":datetime("2010-09-27T18:12:59"),"friend-ids":{{27400643,15449089,802964,45059523,9603951,20911122,46243977,45487995,34528880,16093159,22484957,3951663,12349433,7887502,34786818,13014384,28307526,30476565,7746152,17600641,36877141,4513081,25065078}},"employment":[{"organization-name":"Ontotanin","start-date":date("2012-08-04"),"end-date":date("2012-08-25")}]}
+{"id":10878898,"id-copy":10878898,"alias":"Webster","name":"WebsterCarr","user-since":datetime("2006-07-28T21:17:56"),"user-since-copy":datetime("2006-07-28T21:17:56"),"friend-ids":{{11755002,37594815,4340697,27424145,22193377,31509516,31372689,47386546,30347891,4070454,18531894,28306285,14110568,17830332}},"employment":[{"organization-name":"Medflex","start-date":date("2002-03-12")}]}
+{"id":11943412,"id-copy":11943412,"alias":"Kizzie","name":"KizzieBillimek","user-since":datetime("2011-08-25T09:24:43"),"user-since-copy":datetime("2011-08-25T09:24:43"),"friend-ids":{{47433684,41380366,5933545,6348490,24429719,22579519,21550752,4653838,44131628,7980571,3208666,35631166,13693250,41263305,29172668,24656473,31110672,11323134,23674731,37422602,20327470,13419973}},"employment":[{"organization-name":"Quoline","start-date":date("2012-03-18"),"end-date":date("2012-06-09")}]}
+{"id":10178182,"id-copy":10178182,"alias":"Jen","name":"JenOtis","user-since":datetime("2007-08-09T09:42:29"),"user-since-copy":datetime("2007-08-09T09:42:29"),"friend-ids":{{26278603,27983753,13714345,35452213,27849291,21838200,1008530,27777115,27069057,35804914,34598070,10076890,12795361,16653787,2916026,27047674,8630755,29822673}},"employment":[{"organization-name":"Plexlane","start-date":date("2005-10-10")}]}
+{"id":9291964,"id-copy":9291964,"alias":"Ned","name":"NedPullman","user-since":datetime("2011-02-02T07:25:43"),"user-since-copy":datetime("2011-02-02T07:25:43"),"friend-ids":{{3168566,3349059,43400084,26187570,11222713,9924690,7250860,9801843,18856900,3558502,17237369,20047877,28454433,12279948,19319514,36151797}},"employment":[{"organization-name":"Ontotanin","start-date":date("2001-08-11")}]}
+{"id":10484578,"id-copy":10484578,"alias":"Troy","name":"TroyWheeler","user-since":datetime("2006-12-19T11:23:18"),"user-since-copy":datetime("2006-12-19T11:23:18"),"friend-ids":{{13536585,23059550,16602050,12025612,25014410,13465266}},"employment":[{"organization-name":"Dancode","start-date":date("2011-10-23")}]}
+{"id":11313361,"id-copy":11313361,"alias":"Lashawn","name":"LashawnSchuth","user-since":datetime("2006-08-24T02:37:43"),"user-since-copy":datetime("2006-08-24T02:37:43"),"friend-ids":{{3844342,31605302,11335667,3890958}},"employment":[{"organization-name":"Ganjatax","start-date":date("2001-04-04"),"end-date":date("2006-12-03")}]}
+{"id":11417455,"id-copy":11417455,"alias":"Malka","name":"MalkaWilkinson","user-since":datetime("2012-04-11T17:22:49"),"user-since-copy":datetime("2012-04-11T17:22:49"),"friend-ids":{{29261780,13274200,41060932,8851180,34769837,3296096,19488423,41776348,44518076,16669411,19983817,26799511,16166476,31396373,4090033,37968801,36665813}},"employment":[{"organization-name":"Icerunin","start-date":date("2004-03-12")}]}
+{"id":10323868,"id-copy":10323868,"alias":"Floyd","name":"FloydCostello","user-since":datetime("2007-12-17T05:45:55"),"user-since-copy":datetime("2007-12-17T05:45:55"),"friend-ids":{{16296950,29360254,19980961,43395913,46984972,2696536,9715184,10851075,25657111,46730259,9182621,31950695,46717390,16664917,38439464,6987406,28167105,10608129,11375117,4306430,31737185,29321535,7420588}},"employment":[{"organization-name":"overtech","start-date":date("2000-08-21")}]}
+{"id":9532474,"id-copy":9532474,"alias":"Chester","name":"ChesterAshmore","user-since":datetime("2012-02-03T20:36:34"),"user-since-copy":datetime("2012-02-03T20:36:34"),"friend-ids":{{11340481,15957237,47048138,41603112,6953329,6926093,20866295,329274,16187993,13406075,34601684,46151089,26165473,2882718,20731108}},"employment":[{"organization-name":"Fixdintex","start-date":date("2009-03-14")}]}
+{"id":9146107,"id-copy":9146107,"alias":"Femie","name":"FemieBurns","user-since":datetime("2007-05-05T03:23:12"),"user-since-copy":datetime("2007-05-05T03:23:12"),"friend-ids":{{38688633,2489245,43502175,34373436,11854240,23544813,44263720,20953878,37021620,16028559,20673451,46975172,47409532,44524395}},"employment":[{"organization-name":"Redelectronics","start-date":date("2008-07-26")}]}
+{"id":11659237,"id-copy":11659237,"alias":"Orlando","name":"OrlandoMcloskey","user-since":datetime("2006-09-15T00:02:58"),"user-since-copy":datetime("2006-09-15T00:02:58"),"friend-ids":{{18927260,17411696,20569511,5242025,18974872,24923117,42416784,37339853,42886763,12241986,40609114,8814896,30383771,23631329,41937811,13354366,40113344,11968348,23416173,1546554,46467044,5542363,32084191,3049632}},"employment":[{"organization-name":"zoomplus","start-date":date("2006-04-20")}]}
+{"id":11130676,"id-copy":11130676,"alias":"Krystal","name":"KrystalDavis","user-since":datetime("2008-08-18T00:59:11"),"user-since-copy":datetime("2008-08-18T00:59:11"),"friend-ids":{{44775993,31503397,32012007,16923302,37099907,14276165,40040126,38310068}},"employment":[{"organization-name":"Xx-technology","start-date":date("2003-11-21")}]}
+{"id":10413097,"id-copy":10413097,"alias":"Lindsay","name":"LindsayDoverspike","user-since":datetime("2005-03-24T22:42:49"),"user-since-copy":datetime("2005-03-24T22:42:49"),"friend-ids":{{773762,43764188,23133486,27099138,38010544,38283504,38432745,32450505,34499948,38200436,44093983,41684052,41353940,29027114,2947798,25212070,9522627,18680730,13060818,41586559}},"employment":[{"organization-name":"Hexsanhex","start-date":date("2010-01-23"),"end-date":date("2011-01-14")}]}
+{"id":10585294,"id-copy":10585294,"alias":"Bryan","name":"BryanEliza","user-since":datetime("2005-02-03T16:20:19"),"user-since-copy":datetime("2005-02-03T16:20:19"),"friend-ids":{{6407647,24838863,45997254,42728806,37001718,46932382}},"employment":[{"organization-name":"Tanzumbam","start-date":date("2003-03-15"),"end-date":date("2008-04-24")}]}
+{"id":10503262,"id-copy":10503262,"alias":"Suzanne","name":"SuzanneFonblanque","user-since":datetime("2012-03-16T20:22:06"),"user-since-copy":datetime("2012-03-16T20:22:06"),"friend-ids":{{17868500,500991,7701699,45401842,16746916,24217608,46250003,17567888,28186634}},"employment":[{"organization-name":"Medflex","start-date":date("2001-01-18"),"end-date":date("2005-08-07")}]}
+{"id":10295389,"id-copy":10295389,"alias":"Major","name":"MajorDrabble","user-since":datetime("2009-05-23T12:56:48"),"user-since-copy":datetime("2009-05-23T12:56:48"),"friend-ids":{{}},"employment":[{"organization-name":"Quadlane","start-date":date("2000-10-26")}]}
+{"id":11271517,"id-copy":11271517,"alias":"Amaryllis","name":"AmaryllisNewlove","user-since":datetime("2009-06-10T04:18:11"),"user-since-copy":datetime("2009-06-10T04:18:11"),"friend-ids":{{6594489,17958014,4087759,38993546,1741537,8374107,30133658,33873746}},"employment":[{"organization-name":"Xx-drill","start-date":date("2004-12-21"),"end-date":date("2011-08-19")}]}
+{"id":10754107,"id-copy":10754107,"alias":"Jeri","name":"JeriSanner","user-since":datetime("2009-11-15T23:47:08"),"user-since-copy":datetime("2009-11-15T23:47:08"),"friend-ids":{{19868241,28778419,16761189,28588239,1592484,41256056,36550491,10555328,3086612,37431116,45976270}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2004-11-06")}]}
+{"id":9919033,"id-copy":9919033,"alias":"Bailey","name":"BaileyHay","user-since":datetime("2005-01-06T07:43:18"),"user-since-copy":datetime("2005-01-06T07:43:18"),"friend-ids":{{28198532}},"employment":[{"organization-name":"Kanelectrics","start-date":date("2009-02-08"),"end-date":date("2010-06-08")}]}
+{"id":9239515,"id-copy":9239515,"alias":"Precious","name":"PreciousWeingarten","user-since":datetime("2006-07-03T10:28:56"),"user-since-copy":datetime("2006-07-03T10:28:56"),"friend-ids":{{20459132,9181399,30604442,45266959,31805782,8190732,46444663,46572075,43980715,42547186,21087158,38075989,32228414,25466991,4929897,33467622,35742242,7150399,16997658,18543557,11799062}},"employment":[{"organization-name":"Viatechi","start-date":date("2003-02-15")}]}
+{"id":9512971,"id-copy":9512971,"alias":"Algar","name":"AlgarKepplinger","user-since":datetime("2011-10-11T02:54:01"),"user-since-copy":datetime("2011-10-11T02:54:01"),"friend-ids":{{1076656,1837449,43428033,21710004,41167492,17526252}},"employment":[{"organization-name":"Tranzap","start-date":date("2003-04-26"),"end-date":date("2006-02-24")}]}
+{"id":9069397,"id-copy":9069397,"alias":"Manuel","name":"ManuelTrevithick","user-since":datetime("2009-01-25T00:11:22"),"user-since-copy":datetime("2009-01-25T00:11:22"),"friend-ids":{{1121944,14645273,16100117,45331540,17901062,7344920,22533580,16386626,4267586,34975914,28841442,38737330,31607047,11785331,9617022,44328180,30996836,14315445,18464409,21038654,14409120,12230754,25856707}},"employment":[{"organization-name":"tresline","start-date":date("2011-10-12"),"end-date":date("2011-03-28")}]}
+{"id":10350421,"id-copy":10350421,"alias":"Diane","name":"DianeFisher","user-since":datetime("2010-10-19T11:08:52"),"user-since-copy":datetime("2010-10-19T11:08:52"),"friend-ids":{{22455675,20415125,21917591,44414352,39158851,3446534,6627839,28358200,1176552,37914774}},"employment":[{"organization-name":"Zuncan","start-date":date("2008-06-04"),"end-date":date("2009-09-11")}]}
+{"id":10397017,"id-copy":10397017,"alias":"Holly","name":"HollyHatch","user-since":datetime("2006-04-12T03:26:11"),"user-since-copy":datetime("2006-04-12T03:26:11"),"friend-ids":{{1504006,21411501,20934982,24019384,8634101,25659178,16581112,2481631,15544800}},"employment":[{"organization-name":"linedexon","start-date":date("2000-12-04")}]}
+{"id":10745200,"id-copy":10745200,"alias":"Kaety","name":"KaetyOppenheimer","user-since":datetime("2008-11-21T08:11:11"),"user-since-copy":datetime("2008-11-21T08:11:11"),"friend-ids":{{32006369,4542624,28242708,20936957,11063561,31392192,34444041,754368,37317926}},"employment":[{"organization-name":"Solophase","start-date":date("2010-06-07")}]}
+{"id":10896556,"id-copy":10896556,"alias":"Kimberleigh","name":"KimberleighWoolery","user-since":datetime("2005-05-12T17:22:37"),"user-since-copy":datetime("2005-05-12T17:22:37"),"friend-ids":{{6300953,46149018,25478406,577782,38073266,11461118,10240145,686269,37990652,26865957}},"employment":[{"organization-name":"Ganjatax","start-date":date("2007-05-03")}]}
+{"id":9867190,"id-copy":9867190,"alias":"Elvis","name":"ElvisBasinger","user-since":datetime("2009-01-16T11:48:43"),"user-since-copy":datetime("2009-01-16T11:48:43"),"friend-ids":{{31562017,45465097,29858836,21720764,37465930,20639296,7168709}},"employment":[{"organization-name":"Basecone","start-date":date("2011-01-11"),"end-date":date("2011-01-26")}]}
+{"id":9736855,"id-copy":9736855,"alias":"Sudie","name":"SudieAlbright","user-since":datetime("2011-10-08T08:46:27"),"user-since-copy":datetime("2011-10-08T08:46:27"),"friend-ids":{{20506190,13537252,46211902,4320089}},"employment":[{"organization-name":"Doublezone","start-date":date("2004-12-07"),"end-date":date("2010-07-02")}]}
diff --git a/asterix-app/data/temporal/simpletemp_30.json b/asterix-app/data/temporal/simpletemp_30.json
new file mode 100644
index 0000000..f0a4582
--- /dev/null
+++ b/asterix-app/data/temporal/simpletemp_30.json
@@ -0,0 +1,3 @@
+{ "date": date("4619-11-23"), "time": time("14:29:36.786Z"), "datetime": datetime("2749-01-27T17:27:30.020Z"), "duration": duration("-P474133Y7M854630DT4H40M6.45S"), "year-month-duration": year-month-duration("P193989Y3M"), "day-time-duration": day-time-duration("P4477686DT4H49M31.87S"), "date-interval": interval-date("-9537-08-04, 9656-06-03"), "time-interval": interval-time("12:04:45.689Z, 12:41:59.002Z"), "datetime-interval": interval-datetime("-2640-10-11T17:32:15.675Z, 4104-02-01T05:59:11.902Z") }
+{ "date": date("-9971-09-24"), "time": time("11:38:17.154Z"), "datetime": datetime("1259-11-13T09:49:11.852Z"), "duration": duration("P473653Y9M4566143DT10H20M53.61S"), "year-month-duration": year-month-duration("P148233Y10M"), "day-time-duration": day-time-duration("-P7236357DT2H56M56.164S"), "date-interval": interval-date("-0255-09-06, 4925-05-03"), "time-interval": interval-time("23:10:45.169Z, 01:37:48.736Z"), "datetime-interval": interval-datetime("0534-12-08T08:20:31.487Z, 6778-02-16T22:40:21.653Z") }
+{ "date": date("7986-11-25"), "time": time("12:49:39.736Z"), "datetime": datetime("-8337-01-30T15:23:07.598Z"), "duration": duration("-P184484Y7M2241423DT10H42M49.500S"), "year-month-duration": year-month-duration("-P546031Y3M"), "day-time-duration": day-time-duration("P2623386DT10H32M31.983S"), "date-interval": interval-date("-4514-05-24, 3337-08-26"), "time-interval": interval-time("04:16:42.321Z, 02:22:56.816Z"), "datetime-interval": interval-datetime("2129-12-12T13:18:35.758Z, 8647-07-01T13:10:19.691Z") }
diff --git a/asterix-app/data/tinysocial/fbm.adm b/asterix-app/data/tinysocial/fbm.adm
new file mode 100644
index 0000000..58d7f08
--- /dev/null
+++ b/asterix-app/data/tinysocial/fbm.adm
@@ -0,0 +1,15 @@
+{"message-id":1,"author-id":3,"in-response-to":2,"sender-location":point("47.16,77.75"),"message":" love sprint its shortcut-menu is awesome:)"}
+{"message-id":2,"author-id":1,"in-response-to":4,"sender-location":point("41.66,80.87"),"message":" dislike iphone its touch-screen is horrible"}
+{"message-id":3,"author-id":2,"in-response-to":4,"sender-location":point("48.09,81.01"),"message":" like samsung the plan is amazing"}
+{"message-id":4,"author-id":1,"in-response-to":2,"sender-location":point("37.73,97.04"),"message":" can't stand at&t the network is horrible:("}
+{"message-id":5,"author-id":6,"in-response-to":2,"sender-location":point("34.7,90.76"),"message":" love sprint the customization is mind-blowing"}
+{"message-id":6,"author-id":2,"in-response-to":1,"sender-location":point("31.5,75.56"),"message":" like t-mobile its platform is mind-blowing"}
+{"message-id":7,"author-id":5,"in-response-to":15,"sender-location":point("32.91,85.05"),"message":" dislike sprint the speed is horrible"}
+{"message-id":8,"author-id":1,"in-response-to":11,"sender-location":point("40.33,80.87"),"message":" like verizon the 3G is awesome:)"}
+{"message-id":9,"author-id":3,"in-response-to":12,"sender-location":point("34.45,96.48"),"message":" love verizon its wireless is good"}
+{"message-id":10,"author-id":1,"in-response-to":12,"sender-location":point("42.5,70.01"),"message":" can't stand motorola the touch-screen is terrible"}
+{"message-id":11,"author-id":1,"in-response-to":1,"sender-location":point("38.97,77.49"),"message":" can't stand at&t its plan is terrible"}
+{"message-id":12,"author-id":10,"in-response-to":6,"sender-location":point("42.26,77.76"),"message":" can't stand t-mobile its voicemail-service is OMG:("}
+{"message-id":13,"author-id":10,"in-response-to":4,"sender-location":point("42.77,78.92"),"message":" dislike iphone the voice-command is bad:("}
+{"message-id":14,"author-id":9,"in-response-to":12,"sender-location":point("41.33,85.28"),"message":" love at&t its 3G is good:)"}
+{"message-id":15,"author-id":7,"in-response-to":11,"sender-location":point("44.47,67.11"),"message":" like iphone the voicemail-service is awesome"}
diff --git a/asterix-app/data/tinysocial/fbu.adm b/asterix-app/data/tinysocial/fbu.adm
new file mode 100644
index 0000000..7e99ea4
--- /dev/null
+++ b/asterix-app/data/tinysocial/fbu.adm
@@ -0,0 +1,10 @@
+{"id":1,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]}
+{"id":2,"alias":"Isbel","name":"IsbelDull","user-since":datetime("2011-01-22T10:10:00"),"friend-ids":{{1,4}},"employment":[{"organization-name":"Hexviafind","start-date":date("2010-04-27")}]}
+{"id":3,"alias":"Emory","name":"EmoryUnk","user-since":datetime("2012-07-10T10:10:00"),"friend-ids":{{1,5,8,9}},"employment":[{"organization-name":"geomedia","start-date":date("2010-06-17"),"end-date":date("2010-01-26")}]}
+{"id":4,"alias":"Nicholas","name":"NicholasStroh","user-since":datetime("2010-12-27T10:10:00"),"friend-ids":{{2}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2010-06-08")}]}
+{"id":5,"alias":"Von","name":"VonKemble","user-since":datetime("2010-01-05T10:10:00"),"friend-ids":{{3,6,10}},"employment":[{"organization-name":"Kongreen","start-date":date("2010-11-27")}]}
+{"id":6,"alias":"Willis","name":"WillisWynne","user-since":datetime("2005-01-17T10:10:00"),"friend-ids":{{1,3,7}},"employment":[{"organization-name":"jaydax","start-date":date("2009-05-15")}]}
+{"id":7,"alias":"Suzanna","name":"SuzannaTillson","user-since":datetime("2012-08-07T10:10:00"),"friend-ids":{{6}},"employment":[{"organization-name":"Labzatron","start-date":date("2011-04-19")}]}
+{"id":8,"alias":"Nila","name":"NilaMilliron","user-since":datetime("2008-01-01T10:10:00"),"friend-ids":{{3}},"employment":[{"organization-name":"Plexlane","start-date":date("2010-02-28")}]}
+{"id":9,"alias":"Woodrow","name":"WoodrowNehling","user-since":datetime("2005-09-20T10:10:00"),"friend-ids":{{3,10}},"employment":[{"organization-name":"Zuncan","start-date":date("2003-04-22"),"end-date":date("2009-12-13")}]}
+{"id":10,"alias":"Bram","name":"BramHatch","user-since":datetime("2010-10-16T10:10:00"),"friend-ids":{{1,5,9}},"employment":[{"organization-name":"physcane","start-date":date("2007-06-05"),"end-date":date("2011-11-05")}]}
diff --git a/asterix-app/data/tinysocial/twm.adm b/asterix-app/data/tinysocial/twm.adm
new file mode 100644
index 0000000..fa764af
--- /dev/null
+++ b/asterix-app/data/tinysocial/twm.adm
@@ -0,0 +1,12 @@
+{"tweetid":"1","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("47.44,80.65"),"send-time":datetime("2008-04-26T10:10:00"),"referred-topics":{{"t-mobile","customization"}},"message-text":" love t-mobile its customization is good:)"}
+{"tweetid":"2","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("32.84,67.14"),"send-time":datetime("2010-05-13T10:10:00"),"referred-topics":{{"verizon","shortcut-menu"}},"message-text":" like verizon its shortcut-menu is awesome:)"}
+{"tweetid":"3","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("29.72,75.8"),"send-time":datetime("2006-11-04T10:10:00"),"referred-topics":{{"motorola","speed"}},"message-text":" like motorola the speed is good:)"}
+{"tweetid":"4","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("39.28,70.48"),"send-time":datetime("2011-12-26T10:10:00"),"referred-topics":{{"sprint","voice-command"}},"message-text":" like sprint the voice-command is mind-blowing:)"}
+{"tweetid":"5","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("40.09,92.69"),"send-time":datetime("2006-08-04T10:10:00"),"referred-topics":{{"motorola","speed"}},"message-text":" can't stand motorola its speed is terrible:("}
+{"tweetid":"6","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("47.51,83.99"),"send-time":datetime("2010-05-07T10:10:00"),"referred-topics":{{"iphone","voice-clarity"}},"message-text":" like iphone the voice-clarity is good:)"}
+{"tweetid":"7","user":{"screen-name":"ChangEwing_573","lang":"en","friends_count":182,"statuses_count":394,"name":"Chang Ewing","followers_count":32136},"sender-location":point("36.21,72.6"),"send-time":datetime("2011-08-25T10:10:00"),"referred-topics":{{"samsung","platform"}},"message-text":" like samsung the platform is good"}
+{"tweetid":"8","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("46.05,93.34"),"send-time":datetime("2005-10-14T10:10:00"),"referred-topics":{{"t-mobile","shortcut-menu"}},"message-text":" like t-mobile the shortcut-menu is awesome:)"}
+{"tweetid":"9","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("36.86,74.62"),"send-time":datetime("2012-07-21T10:10:00"),"referred-topics":{{"verizon","voicemail-service"}},"message-text":" love verizon its voicemail-service is awesome"}
+{"tweetid":"10","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("29.15,76.53"),"send-time":datetime("2008-01-26T10:10:00"),"referred-topics":{{"verizon","voice-clarity"}},"message-text":" hate verizon its voice-clarity is OMG:("}
+{"tweetid":"11","user":{"screen-name":"NilaMilliron_tw","lang":"en","friends_count":445,"statuses_count":164,"name":"Nila Milliron","followers_count":22649},"sender-location":point("37.59,68.42"),"send-time":datetime("2008-03-09T10:10:00"),"referred-topics":{{"iphone","platform"}},"message-text":" can't stand iphone its platform is terrible"}
+{"tweetid":"12","user":{"screen-name":"OliJackson_512","lang":"en","friends_count":445,"statuses_count":164,"name":"Oli Jackson","followers_count":22649},"sender-location":point("24.82,94.63"),"send-time":datetime("2010-02-13T10:10:00"),"referred-topics":{{"samsung","voice-command"}},"message-text":" like samsung the voice-command is amazing:)"}
diff --git a/asterix-app/data/tinysocial/twu.adm b/asterix-app/data/tinysocial/twu.adm
new file mode 100644
index 0000000..32a1917
--- /dev/null
+++ b/asterix-app/data/tinysocial/twu.adm
@@ -0,0 +1,4 @@
+{"screen-name":"NathanGiesen@211","lang":"en","friends_count":18,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416}
+{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159}
+{"screen-name":"NilaMilliron_tw","lang":"en","friends_count":445,"statuses_count":164,"name":"Nila Milliron","followers_count":22649}
+{"screen-name":"ChangEwing_573","lang":"en","friends_count":182,"statuses_count":394,"name":"Chang Ewing","followers_count":32136}
diff --git a/asterix-app/pom.xml b/asterix-app/pom.xml
index 15af3bc..de5558d 100644
--- a/asterix-app/pom.xml
+++ b/asterix-app/pom.xml
@@ -100,9 +100,7 @@
 		<dependency>
 			<groupId>javax.servlet</groupId>
 			<artifactId>servlet-api</artifactId>
-			<version>2.5</version>
 			<type>jar</type>
-			<scope>compile</scope>
 		</dependency>
 		<dependency>
 			<groupId>org.eclipse.jetty</groupId>
@@ -182,6 +180,12 @@
                         <type>test-jar</type>
                         <scope>test</scope>
                 </dependency>
+                <dependency>
+                        <groupId>edu.uci.ics.asterix</groupId>
+                        <artifactId>asterix-transactions</artifactId>
+                        <version>0.0.6-SNAPSHOT</version>
+                        <scope>compile</scope>
+                </dependency>
 		<dependency>
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-core</artifactId>
@@ -210,11 +214,6 @@
 			<scope>test</scope>
 		</dependency>
                 <dependency>
-                    <groupId>net.sourceforge.cobertura</groupId>
-                    <artifactId>cobertura</artifactId>
-                    <version>1.9.4</version>
-                </dependency>
-                <dependency>
                 	<groupId>edu.uci.ics.asterix</groupId>
                 	<artifactId>asterix-test-framework</artifactId>
                 	<version>0.0.6-SNAPSHOT</version>
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
index e1114eb..d5d7409 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
@@ -26,9 +26,9 @@
 import edu.uci.ics.asterix.aql.expression.Query;
 import edu.uci.ics.asterix.aql.expression.visitor.AQLPrintVisitor;
 import edu.uci.ics.asterix.aql.rewrites.AqlRewriter;
-import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
 import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.dataflow.data.common.AqlExpressionTypeComputer;
 import edu.uci.ics.asterix.dataflow.data.common.AqlMergeAggregationExpressionFactory;
@@ -40,9 +40,9 @@
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.asterix.optimizer.base.RuleCollections;
 import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
 import edu.uci.ics.asterix.translator.AqlExpressionToPlanTranslator;
 import edu.uci.ics.asterix.translator.CompiledStatements.ICompiledDmlStatement;
@@ -136,12 +136,12 @@
         }
 
         @Override
-        public IOptimizationContext createOptimizationContext(int varCounter, int frameSize,
+        public IOptimizationContext createOptimizationContext(int varCounter,
                 IExpressionEvalSizeComputer expressionEvalSizeComputer,
                 IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
                 IExpressionTypeComputer expressionTypeComputer, INullableTypeComputer nullableTypeComputer,
                 PhysicalOptimizationConfig physicalOptimizationConfig) {
-            return new AlgebricksOptimizationContext(varCounter, frameSize, expressionEvalSizeComputer,
+            return new AlgebricksOptimizationContext(varCounter, expressionEvalSizeComputer,
                     mergeAggregationExpressionFactory, expressionTypeComputer, nullableTypeComputer,
                     physicalOptimizationConfig);
         }
@@ -162,7 +162,7 @@
             out.println();
             switch (pdf) {
                 case HTML: {
-                    out.println("<h3>Expression tree:</h3>");
+                    out.println("<h4>Expression tree:</h4>");
                     out.println("<pre>");
                     break;
                 }
@@ -197,7 +197,7 @@
 
             switch (pdf) {
                 case HTML: {
-                    out.println("<h3>Rewriten expression tree:</h3>");
+                    out.println("<h4>Rewritten expression tree:</h4>");
                     out.println("<pre>");
                     break;
                 }
@@ -220,7 +220,7 @@
 
         }
 
-        edu.uci.ics.asterix.transaction.management.service.transaction.JobId asterixJobId = JobIdFactory
+        edu.uci.ics.asterix.common.transactions.JobId asterixJobId = JobIdFactory
                 .generateJobId();
         queryMetadataProvider.setJobId(asterixJobId);
         AqlExpressionToPlanTranslator t = new AqlExpressionToPlanTranslator(queryMetadataProvider, varCounter,
@@ -234,7 +234,7 @@
 
             switch (pdf) {
                 case HTML: {
-                    out.println("<h3>Logical plan:</h3>");
+                    out.println("<h4>Logical plan:</h4>");
                     out.println("<pre>");
                     break;
                 }
@@ -260,22 +260,26 @@
 
         AsterixCompilerProperties compilerProperties = AsterixAppContextInfo.getInstance().getCompilerProperties();
         int frameSize = compilerProperties.getFrameSize();
+        int sortFrameLimit = (int) (compilerProperties.getSortMemorySize() / frameSize);
+        int joinFrameLimit = (int) (compilerProperties.getJoinMemorySize() / frameSize);
+        OptimizationConfUtil.getPhysicalOptimizationConfig().setFrameSize(frameSize);
+        OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesExternalSort(sortFrameLimit);
+        OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesHybridHash(joinFrameLimit);
+        
 
         HeuristicCompilerFactoryBuilder builder = new HeuristicCompilerFactoryBuilder(
                 AqlOptimizationContextFactory.INSTANCE);
+        builder.setPhysicalOptimizationConfig(OptimizationConfUtil.getPhysicalOptimizationConfig());
         builder.setLogicalRewrites(buildDefaultLogicalRewrites());
         builder.setPhysicalRewrites(buildDefaultPhysicalRewrites());
         IDataFormat format = queryMetadataProvider.getFormat();
         ICompilerFactory compilerFactory = builder.create();
-        builder.setFrameSize(frameSize);
         builder.setExpressionEvalSizeComputer(format.getExpressionEvalSizeComputer());
         builder.setIMergeAggregationExpressionFactory(new AqlMergeAggregationExpressionFactory());
         builder.setPartialAggregationTypeComputer(new AqlPartialAggregationTypeComputer());
         builder.setExpressionTypeComputer(AqlExpressionTypeComputer.INSTANCE);
         builder.setNullableTypeComputer(AqlNullableTypeComputer.INSTANCE);
 
-        OptimizationConfUtil.getPhysicalOptimizationConfig().setFrameSize(frameSize);
-        builder.setPhysicalOptimizationConfig(OptimizationConfUtil.getPhysicalOptimizationConfig());
 
         ICompiler compiler = compilerFactory.createCompiler(plan, queryMetadataProvider, t.getVarCounter());
         if (pc.isOptimize()) {
@@ -289,7 +293,7 @@
                 } else {
                     switch (pdf) {
                         case HTML: {
-                            out.println("<h3>Optimized logical plan:</h3>");
+                            out.println("<h4>Optimized logical plan:</h4>");
                             out.println("<pre>");
                             break;
                         }
@@ -327,6 +331,7 @@
         builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
         builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
         builder.setNullWriterFactory(format.getNullWriterFactory());
+        builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
 
         switch (pdf) {
             case JSON:
@@ -348,7 +353,7 @@
         if (pc.isPrintJob()) {
             switch (pdf) {
                 case HTML: {
-                    out.println("<h3>Hyracks job:</h3>");
+                    out.println("<h4>Hyracks job:</h4>");
                     out.println("<pre>");
                     break;
                 }
@@ -380,7 +385,7 @@
             hcc.waitForCompletion(jobId);
             long endTime = System.currentTimeMillis();
             double duration = (endTime - startTime) / 1000.00;
-            out.println("<pre>Duration: " + duration + "</pre>");
+            out.println("<pre>Duration: " + duration + " sec</pre>");
         }
 
     }
@@ -402,7 +407,7 @@
             }
             long endTime = System.currentTimeMillis();
             double duration = (endTime - startTime) / 1000.00;
-            out.println("<pre>Duration: " + duration + "</pre>");
+            out.println("<pre>Duration: " + duration + " sec</pre>");
         }
 
     }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
similarity index 70%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
rename to asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
index 8fe3341..9c04b80 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
@@ -1,8 +1,9 @@
-package edu.uci.ics.asterix.common.context;
+package edu.uci.ics.asterix.api.common;
 
 import java.io.IOException;
 import java.util.logging.Logger;
 
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
 import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
 import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
@@ -10,26 +11,27 @@
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
+import edu.uci.ics.asterix.common.context.AsterixFileMapManager;
+import edu.uci.ics.asterix.common.context.ConstantMergePolicy;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTrackerFactory;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepositoryFactory;
-import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
+import edu.uci.ics.hyracks.api.lifecycle.LifeCycleComponentManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
@@ -46,7 +48,9 @@
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
 
-public class AsterixAppRuntimeContext implements IAsterixPropertiesProvider {
+public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext, IAsterixPropertiesProvider {
+    private static final int METADATA_IO_DEVICE_ID = 0;
+
     private final INCApplicationContext ncApplicationContext;
 
     private AsterixCompilerProperties compilerProperties;
@@ -55,17 +59,14 @@
     private AsterixStorageProperties storageProperties;
     private AsterixTransactionProperties txnProperties;
 
-    private IIndexLifecycleManager indexLifecycleManager;
+    private DatasetLifecycleManager indexLifecycleManager;
     private IFileMapManager fileMapManager;
     private IBufferCache bufferCache;
-    private TransactionSubsystem txnSubsystem;
+    private ITransactionSubsystem txnSubsystem;
 
     private ILSMMergePolicy mergePolicy;
-    private ILSMOperationTrackerFactory lsmBTreeOpTrackerFactory;
-    private ILSMOperationTrackerFactory lsmRTreeOpTrackerFactory;
-    private ILSMOperationTrackerFactory lsmInvertedIndexOpTrackerFactory;
     private ILSMIOOperationScheduler lsmIOScheduler;
-    private PersistentLocalResourceRepository localResourceRepository;
+    private ILocalResourceRepository localResourceRepository;
     private ResourceIdFactory resourceIdFactory;
     private IIOManager ioManager;
     private boolean isShuttingdown;
@@ -91,27 +92,29 @@
         ioManager = ncApplicationContext.getRootContext().getIOManager();
         bufferCache = new BufferCache(ioManager, allocator, prs, pcp, fileMapManager,
                 storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages(),
-                storageProperties.getBufferCacheMaxOpenFiles());
-
-        indexLifecycleManager = new IndexLifecycleManager(storageProperties.getMemoryComponentGlobalBudget());
+                storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory());
 
         lsmIOScheduler = SynchronousScheduler.INSTANCE;
         mergePolicy = new ConstantMergePolicy(storageProperties.getLSMIndexMergeThreshold(), this);
-        lsmBTreeOpTrackerFactory = new IndexOperationTrackerFactory(LSMBTreeIOOperationCallbackFactory.INSTANCE);
-        lsmRTreeOpTrackerFactory = new IndexOperationTrackerFactory(LSMRTreeIOOperationCallbackFactory.INSTANCE);
-        lsmInvertedIndexOpTrackerFactory = new IndexOperationTrackerFactory(
-                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE);
 
         ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
                 ioManager);
         localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
                 .createRepository();
         resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
-
+        indexLifecycleManager = new DatasetLifecycleManager(storageProperties, localResourceRepository);
         IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProviderForRecovery(
                 this);
-        txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider);
+        txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider, txnProperties);
         isShuttingdown = false;
+
+        // The order of registration is important. The buffer cache must registered before recovery and transaction managers.
+        LifeCycleComponentManager.INSTANCE.register((ILifeCycleComponent) bufferCache);
+        LifeCycleComponentManager.INSTANCE.register((ILifeCycleComponent) indexLifecycleManager);
+        LifeCycleComponentManager.INSTANCE.register((ILifeCycleComponent) txnSubsystem.getTransactionManager());
+        LifeCycleComponentManager.INSTANCE.register((ILifeCycleComponent) txnSubsystem.getLogManager());
+        LifeCycleComponentManager.INSTANCE.register((ILifeCycleComponent) txnSubsystem.getLockManager());
+        LifeCycleComponentManager.INSTANCE.register((ILifeCycleComponent) txnSubsystem.getRecoveryManager());
     }
 
     public boolean isShuttingdown() {
@@ -137,7 +140,7 @@
         return fileMapManager;
     }
 
-    public TransactionSubsystem getTransactionSubsystem() {
+    public ITransactionSubsystem getTransactionSubsystem() {
         return txnSubsystem;
     }
 
@@ -153,34 +156,6 @@
         return storageProperties.getBloomFilterFalsePositiveRate();
     }
 
-    public ILSMOperationTrackerFactory getLSMBTreeOperationTrackerFactory() {
-        return lsmBTreeOpTrackerFactory;
-    }
-
-    public ILSMOperationTrackerFactory getLSMRTreeOperationTrackerFactory() {
-        return lsmRTreeOpTrackerFactory;
-    }
-
-    public ILSMOperationTrackerFactory getLSMInvertedIndexOperationTrackerFactory() {
-        return lsmInvertedIndexOpTrackerFactory;
-    }
-
-    public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider() {
-        return AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER;
-    }
-
-    public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider() {
-        return AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER;
-    }
-
-    public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider() {
-        return AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER;
-    }
-
-    public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider() {
-        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
-    }
-
     public ILSMIOOperationScheduler getLSMIOScheduler() {
         return lsmIOScheduler;
     }
@@ -197,6 +172,10 @@
         return ioManager;
     }
 
+    public int getMetaDataIODeviceId() {
+        return METADATA_IO_DEVICE_ID;
+    }
+
     @Override
     public AsterixStorageProperties getStorageProperties() {
         return storageProperties;
@@ -221,4 +200,14 @@
     public AsterixExternalProperties getExternalProperties() {
         return externalProperties;
     }
+
+    @Override
+    public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
+        return indexLifecycleManager.getVirtualBufferCache(datasetID);
+    }
+
+    @Override
+    public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID) {
+        return indexLifecycleManager.getOperationTracker(datasetID);
+    }
 }
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContextProviderForRecovery.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
similarity index 72%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContextProviderForRecovery.java
rename to asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
index 10660cc..1035f0c 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContextProviderForRecovery.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
@@ -1,13 +1,14 @@
-package edu.uci.ics.asterix.common.context;
+package edu.uci.ics.asterix.api.common;
 
-import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
@@ -32,7 +33,7 @@
     }
 
     @Override
-    public TransactionSubsystem getTransactionSubsystem() {
+    public ITransactionSubsystem getTransactionSubsystem() {
         return asterixAppRuntimeContext.getTransactionSubsystem();
     }
 
@@ -72,37 +73,36 @@
     }
 
     @Override
-    public ILSMOperationTrackerFactory getLSMBTreeOperationTrackerFactory() {
-        return asterixAppRuntimeContext.getLSMBTreeOperationTrackerFactory();
-    }
-
-    @Override
-    public ILSMOperationTrackerFactory getLSMRTreeOperationTrackerFactory() {
-        return asterixAppRuntimeContext.getLSMRTreeOperationTrackerFactory();
-    }
-
-    @Override
-    public ILSMOperationTrackerFactory getLSMInvertedIndexOperationTrackerFactory() {
-        return asterixAppRuntimeContext.getLSMInvertedIndexOperationTrackerFactory();
-    }
-
-    @Override
-    public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider() {
-        return asterixAppRuntimeContext.getLSMBTreeIOOperationCallbackProvider();
-    }
-
-    @Override
-    public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider() {
-        return asterixAppRuntimeContext.getLSMRTreeIOOperationCallbackProvider();
-    }
-
-    @Override
-    public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider() {
-        return asterixAppRuntimeContext.getLSMInvertedIndexIOOperationCallbackProvider();
+    public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
+        return asterixAppRuntimeContext.getVirtualBufferCache(datasetID);
     }
 
     @Override
     public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider() {
-        return asterixAppRuntimeContext.getNoOpIOOperationCallbackProvider();
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider() {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider() {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider() {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID) {
+        return asterixAppRuntimeContext.getLSMBTreeOperationTracker(datasetID);
     }
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java
index 272050f..fb87a8e1 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -1,5 +1,6 @@
 package edu.uci.ics.asterix.api.common;
 
+import java.io.File;
 import java.util.EnumSet;
 
 import edu.uci.ics.asterix.common.config.GlobalConfig;
@@ -50,6 +51,8 @@
         ncConfig1.datasetIPAddress = "127.0.0.1";
         ncConfig1.resultHistorySize = 1000;
         ncConfig1.nodeId = NC1_ID;
+        ncConfig1.ioDevices = System.getProperty("java.io.tmpdir") + File.separator + "nc1/iodevice0" + ","
+                + System.getProperty("java.io.tmpdir") + File.separator + "nc1/iodevice1";
         ncConfig1.appNCMainClass = NCApplicationEntryPoint.class.getName();
         nc1 = new NodeControllerService(ncConfig1);
         nc1.start();
@@ -62,6 +65,8 @@
         ncConfig2.datasetIPAddress = "127.0.0.1";
         ncConfig2.resultHistorySize = 1000;
         ncConfig2.nodeId = NC2_ID;
+        ncConfig2.ioDevices = System.getProperty("java.io.tmpdir") + File.separator + "nc2/iodevice0" + ","
+                + System.getProperty("java.io.tmpdir") + File.separator + "nc2/iodevice1";
         ncConfig2.appNCMainClass = NCApplicationEntryPoint.class.getName();
         nc2 = new NodeControllerService(ncConfig2);
         nc2.start();
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java
index 12fa51a..bb03dc3 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java
@@ -1,13 +1,16 @@
 package edu.uci.ics.asterix.api.http.servlet;
 
+import java.awt.image.BufferedImage;
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.io.OutputStream;
 import java.io.PrintWriter;
 import java.util.List;
 import java.util.logging.Level;
 
+import javax.imageio.ImageIO;
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
@@ -23,6 +26,7 @@
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.result.ResultReader;
+import edu.uci.ics.asterix.result.ResultUtils;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
 import edu.uci.ics.hyracks.client.dataset.HyracksDataset;
@@ -78,40 +82,21 @@
             aqlTranslator.compileAndExecute(hcc, hds, false);
             long endTime = System.currentTimeMillis();
             duration = (endTime - startTime) / 1000.00;
-            out.println("<PRE>Duration of all jobs: " + duration + "</PRE>");
+            out.println("<PRE>Duration of all jobs: " + duration + " sec</PRE>");
         } catch (ParseException | TokenMgrError | edu.uci.ics.asterix.aqlplus.parser.TokenMgrError pe) {
-            out.println("<pre class=\"error\">");
-            String message = pe.getMessage();
-            message = message.replace("<", "&lt");
-            message = message.replace(">", "&gt");
-            out.println("SyntaxError:" + message);
-            int pos = message.indexOf("line");
-            if (pos > 0) {
-                int columnPos = message.indexOf(",", pos + 1 + "line".length());
-                int lineNo = Integer.parseInt(message.substring(pos + "line".length() + 1, columnPos));
-                String[] lines = query.split("\n");
-                if (lineNo >= lines.length) {
-                    out.println("===> &ltBLANK LINE&gt");
-                } else {
-                    String line = lines[lineNo - 1];
-                    out.println("==> " + line);
-                }
-            }
-            out.println("</pre>");
+            GlobalConfig.ASTERIX_LOGGER.log(Level.INFO, pe.toString(), pe);
+            ResultUtils.webUIParseExceptionHandler(out, pe, query);
         } catch (Exception e) {
             GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
-            out.println("<pre class=\"error\">");
-            out.println(e.getMessage());
-            out.println("</pre>");
+            ResultUtils.webUIErrorHandler(out, e);
         }
     }
 
     @Override
     public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
         String resourcePath = null;
-        response.setCharacterEncoding("utf-8");
-        PrintWriter out = response.getWriter();
         String requestURI = request.getRequestURI();
+
         if (requestURI.equals("/")) {
             response.setContentType("text/html");
             resourcePath = "/webui/querytemplate.html";
@@ -124,6 +109,21 @@
             response.sendError(HttpServletResponse.SC_NOT_FOUND);
             return;
         }
+
+        // Special handler for font files and .png resources
+        if (resourcePath.endsWith(".png")) {
+
+            BufferedImage img = ImageIO.read(is);
+            OutputStream outputStream = response.getOutputStream();
+            String formatName = "png";
+            response.setContentType("image/png");
+            ImageIO.write(img, formatName, outputStream);
+            outputStream.close();
+            return;
+
+        }
+
+        response.setCharacterEncoding("utf-8");
         InputStreamReader isr = new InputStreamReader(is);
         StringBuilder sb = new StringBuilder();
         BufferedReader br = new BufferedReader(isr);
@@ -134,6 +134,7 @@
             line = br.readLine();
         }
 
+        PrintWriter out = response.getWriter();
         out.println(sb.toString());
     }
 
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
index 2c15578..8caf05e 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
@@ -32,6 +32,7 @@
 import edu.uci.ics.asterix.aql.base.Statement.Kind;
 import edu.uci.ics.asterix.aql.parser.AQLParser;
 import edu.uci.ics.asterix.aql.parser.ParseException;
+import edu.uci.ics.asterix.aql.parser.TokenMgrError;
 import edu.uci.ics.asterix.aql.translator.AqlTranslator;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -95,36 +96,24 @@
             AqlTranslator aqlTranslator = new AqlTranslator(aqlStatements, out, sessionConfig, format);
 
             aqlTranslator.compileAndExecute(hcc, hds, asyncResults);
-
-        } catch (ParseException pe) {
-            GlobalConfig.ASTERIX_LOGGER.log(Level.INFO, pe.getMessage(), pe);
-            StringBuilder errorMessage = new StringBuilder();
-            String message = pe.getMessage();
-            message = message.replace("<", "&lt");
-            message = message.replace(">", "&gt");
-            errorMessage.append("SyntaxError:" + message + "\n");
-            int pos = message.indexOf("line");
-            if (pos > 0) {
-                int columnPos = message.indexOf(",", pos + 1 + "line".length());
-                int lineNo = Integer.parseInt(message.substring(pos + "line".length() + 1, columnPos));
-                String line = query.split("\n")[lineNo - 1];
-                errorMessage.append("==> " + line + "\n");
-            }
-            JSONObject errorResp = ResultUtils.getErrorResponse(2, errorMessage.toString());
+        } catch (ParseException | TokenMgrError | edu.uci.ics.asterix.aqlplus.parser.TokenMgrError pe) {
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, pe.getMessage(), pe);
+            String errorMessage = ResultUtils.buildParseExceptionMessage(pe, query);
+            JSONObject errorResp = ResultUtils.getErrorResponse(2, errorMessage, "", "");
             out.write(errorResp.toString());
+            response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
         } catch (Exception e) {
-            GlobalConfig.ASTERIX_LOGGER.log(Level.INFO, e.getMessage(), e);
-            StringBuilder errorMessage = new StringBuilder();
-            errorMessage.append(e.getMessage());
-            JSONObject errorResp = ResultUtils.getErrorResponse(99, errorMessage.toString());
-            out.write(errorResp.toString());
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
+            ResultUtils.apiErrorHandler(out, e);
+            response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
         }
     }
 
     private boolean checkForbiddenStatements(List<Statement> aqlStatements, PrintWriter out) {
         for (Statement st : aqlStatements) {
             if (!getAllowedStatements().contains(st.getKind())) {
-                JSONObject errorResp = ResultUtils.getErrorResponse(1, String.format(getErrorMessage(), st.getKind()));
+                JSONObject errorResp = ResultUtils.getErrorResponse(1, String.format(getErrorMessage(), st.getKind()),
+                        "", "");
                 out.write(errorResp.toString());
                 return true;
             }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
index cb786d7..0fbe9fa 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
@@ -9,6 +9,11 @@
 import edu.uci.ics.asterix.api.common.Job;
 import edu.uci.ics.asterix.api.common.SessionConfig;
 import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.translator.AqlTranslator;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.asterix.aql.parser.AQLParser;
 import edu.uci.ics.asterix.aql.parser.ParseException;
 import edu.uci.ics.asterix.aql.translator.AqlTranslator;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index d4e5e22..256c40b 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
@@ -62,6 +62,7 @@
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.file.DatasetOperations;
@@ -90,7 +91,6 @@
 import edu.uci.ics.asterix.om.types.TypeSignature;
 import edu.uci.ics.asterix.result.ResultReader;
 import edu.uci.ics.asterix.result.ResultUtils;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import edu.uci.ics.asterix.translator.AbstractAqlTranslator;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledBeginFeedStatement;
@@ -186,112 +186,107 @@
             metadataProvider.setOutputFile(outputFile);
             metadataProvider.setConfig(config);
             jobsToExecute.clear();
-            try {
-                switch (stmt.getKind()) {
-                    case SET: {
-                        handleSetStatement(metadataProvider, stmt, config);
-                        break;
-                    }
-                    case DATAVERSE_DECL: {
-                        activeDefaultDataverse = handleUseDataverseStatement(metadataProvider, stmt);
-                        break;
-                    }
-                    case CREATE_DATAVERSE: {
-                        handleCreateDataverseStatement(metadataProvider, stmt);
-                        break;
-                    }
-                    case DATASET_DECL: {
-                        handleCreateDatasetStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-                    case CREATE_INDEX: {
-                        handleCreateIndexStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-                    case TYPE_DECL: {
-                        handleCreateTypeStatement(metadataProvider, stmt);
-                        break;
-                    }
-                    case NODEGROUP_DECL: {
-                        handleCreateNodeGroupStatement(metadataProvider, stmt);
-                        break;
-                    }
-                    case DATAVERSE_DROP: {
-                        handleDataverseDropStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-                    case DATASET_DROP: {
-                        handleDatasetDropStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-                    case INDEX_DROP: {
-                        handleIndexDropStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-                    case TYPE_DROP: {
-                        handleTypeDropStatement(metadataProvider, stmt);
-                        break;
-                    }
-                    case NODEGROUP_DROP: {
-                        handleNodegroupDropStatement(metadataProvider, stmt);
-                        break;
-                    }
-
-                    case CREATE_FUNCTION: {
-                        handleCreateFunctionStatement(metadataProvider, stmt);
-                        break;
-                    }
-
-                    case FUNCTION_DROP: {
-                        handleFunctionDropStatement(metadataProvider, stmt);
-                        break;
-                    }
-
-                    case LOAD_FROM_FILE: {
-                        handleLoadFromFileStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-                    case WRITE_FROM_QUERY_RESULT: {
-                        handleWriteFromQueryResultStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-                    case INSERT: {
-                        handleInsertStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-                    case DELETE: {
-                        handleDeleteStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-
-                    case BEGIN_FEED: {
-                        handleBeginFeedStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-
-                    case CONTROL_FEED: {
-                        handleControlFeedStatement(metadataProvider, stmt, hcc);
-                        break;
-                    }
-
-                    case QUERY: {
-                        metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
-                        executionResult.add(handleQuery(metadataProvider, (Query) stmt, hcc, hdc, asyncResults));
-                        break;
-                    }
-
-                    case WRITE: {
-                        Pair<IAWriterFactory, FileSplit> result = handleWriteStatement(metadataProvider, stmt);
-                        if (result.first != null) {
-                            writerFactory = result.first;
-                        }
-                        outputFile = result.second;
-                        break;
-                    }
-
+            switch (stmt.getKind()) {
+                case SET: {
+                    handleSetStatement(metadataProvider, stmt, config);
+                    break;
                 }
-            } catch (Exception e) {
-                throw new AlgebricksException(e);
+                case DATAVERSE_DECL: {
+                    activeDefaultDataverse = handleUseDataverseStatement(metadataProvider, stmt);
+                    break;
+                }
+                case CREATE_DATAVERSE: {
+                    handleCreateDataverseStatement(metadataProvider, stmt);
+                    break;
+                }
+                case DATASET_DECL: {
+                    handleCreateDatasetStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+                case CREATE_INDEX: {
+                    handleCreateIndexStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+                case TYPE_DECL: {
+                    handleCreateTypeStatement(metadataProvider, stmt);
+                    break;
+                }
+                case NODEGROUP_DECL: {
+                    handleCreateNodeGroupStatement(metadataProvider, stmt);
+                    break;
+                }
+                case DATAVERSE_DROP: {
+                    handleDataverseDropStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+                case DATASET_DROP: {
+                    handleDatasetDropStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+                case INDEX_DROP: {
+                    handleIndexDropStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+                case TYPE_DROP: {
+                    handleTypeDropStatement(metadataProvider, stmt);
+                    break;
+                }
+                case NODEGROUP_DROP: {
+                    handleNodegroupDropStatement(metadataProvider, stmt);
+                    break;
+                }
+
+                case CREATE_FUNCTION: {
+                    handleCreateFunctionStatement(metadataProvider, stmt);
+                    break;
+                }
+
+                case FUNCTION_DROP: {
+                    handleFunctionDropStatement(metadataProvider, stmt);
+                    break;
+                }
+
+                case LOAD_FROM_FILE: {
+                    handleLoadFromFileStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+                case WRITE_FROM_QUERY_RESULT: {
+                    handleWriteFromQueryResultStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+                case INSERT: {
+                    handleInsertStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+                case DELETE: {
+                    handleDeleteStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+
+                case BEGIN_FEED: {
+                    handleBeginFeedStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+
+                case CONTROL_FEED: {
+                    handleControlFeedStatement(metadataProvider, stmt, hcc);
+                    break;
+                }
+
+                case QUERY: {
+                    metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
+                    executionResult.add(handleQuery(metadataProvider, (Query) stmt, hcc, hdc, asyncResults));
+                    break;
+                }
+
+                case WRITE: {
+                    Pair<IAWriterFactory, FileSplit> result = handleWriteStatement(metadataProvider, stmt);
+                    if (result.first != null) {
+                        writerFactory = result.first;
+                    }
+                    outputFile = result.second;
+                    break;
+                }
             }
         }
         return executionResult;
@@ -364,7 +359,7 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -384,11 +379,7 @@
         Dataset dataset = null;
         try {
             DatasetDecl dd = (DatasetDecl) stmt;
-            dataverseName = dd.getDataverse() != null ? dd.getDataverse().getValue()
-                    : activeDefaultDataverse != null ? activeDefaultDataverse.getDataverseName() : null;
-            if (dataverseName == null) {
-                throw new AlgebricksException(" dataverse not specified ");
-            }
+            dataverseName = getActiveDataverseName(dd.getDataverse());
             datasetName = dd.getName().getValue();
 
             DatasetType dsType = dd.getDatasetType();
@@ -532,7 +523,7 @@
                 }
             }
 
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -551,16 +542,13 @@
         String datasetName = null;
         String indexName = null;
         JobSpecification spec = null;
+        Dataset ds = null;
         try {
             CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
-            dataverseName = stmtCreateIndex.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : stmtCreateIndex.getDataverseName().getValue();
-            if (dataverseName == null) {
-                throw new AlgebricksException(" dataverse not specified ");
-            }
+            dataverseName = getActiveDataverseName(stmtCreateIndex.getDataverseName());
             datasetName = stmtCreateIndex.getDatasetName().getValue();
 
-            Dataset ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
+            ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
                     datasetName);
             if (ds == null) {
                 throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
@@ -646,7 +634,8 @@
                 metadataProvider.setMetadataTxnContext(mdTxnCtx);
                 CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
                 try {
-                    JobSpecification jobSpec = IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider);
+                    JobSpecification jobSpec = IndexOperations
+                            .buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds);
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     bActiveTxn = false;
 
@@ -672,7 +661,7 @@
                             + "." + datasetName + "." + indexName + ") couldn't be removed from the metadata", e);
                 }
             }
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -686,15 +675,11 @@
 
         try {
             TypeDecl stmtCreateType = (TypeDecl) stmt;
-            String dataverseName = stmtCreateType.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : stmtCreateType.getDataverseName().getValue();
-            if (dataverseName == null) {
-                throw new AlgebricksException(" dataverse not specified ");
-            }
+            String dataverseName = getActiveDataverseName(stmtCreateType.getDataverseName());
             String typeName = stmtCreateType.getIdent().getValue();
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
             if (dv == null) {
-                throw new AlgebricksException("Unknonw dataverse " + dataverseName);
+                throw new AlgebricksException("Unknown dataverse " + dataverseName);
             }
             Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
             if (dt != null) {
@@ -715,7 +700,7 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -759,7 +744,8 @@
                         if (indexes.get(k).isSecondaryIndex()) {
                             CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
                                     indexes.get(k).getIndexName());
-                            jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
+                            jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider,
+                                    datasets.get(j)));
                         }
                     }
 
@@ -829,7 +815,7 @@
                 }
             }
 
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -849,11 +835,7 @@
         List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
         try {
             DropStatement stmtDelete = (DropStatement) stmt;
-            dataverseName = stmtDelete.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : stmtDelete.getDataverseName().getValue();
-            if (dataverseName == null) {
-                throw new AlgebricksException(" dataverse not specified ");
-            }
+            dataverseName = getActiveDataverseName(stmtDelete.getDataverseName());
             datasetName = stmtDelete.getDatasetName().getValue();
 
             Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
@@ -875,7 +857,7 @@
                     if (indexes.get(j).isSecondaryIndex()) {
                         CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
                                 indexes.get(j).getIndexName());
-                        jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
+                        jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
                     }
                 }
                 CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
@@ -938,7 +920,7 @@
                 }
             }
 
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -960,11 +942,7 @@
         try {
             IndexDropStatement stmtIndexDrop = (IndexDropStatement) stmt;
             datasetName = stmtIndexDrop.getDatasetName().getValue();
-            dataverseName = stmtIndexDrop.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : stmtIndexDrop.getDataverseName().getValue();
-            if (dataverseName == null) {
-                throw new AlgebricksException(" dataverse not specified ");
-            }
+            dataverseName = getActiveDataverseName(stmtIndexDrop.getDataverseName());
 
             Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
             if (ds == null) {
@@ -985,7 +963,7 @@
                 }
                 //#. prepare a job to drop the index in NC.
                 CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
-                jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
+                jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider, ds));
 
                 //#. mark PendingDropOp on the existing index
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
@@ -1047,7 +1025,7 @@
                 }
             }
 
-            throw new AlgebricksException(e);
+            throw e;
 
         } finally {
             releaseWriteLatch();
@@ -1062,11 +1040,7 @@
 
         try {
             TypeDropStatement stmtTypeDrop = (TypeDropStatement) stmt;
-            String dataverseName = stmtTypeDrop.getDataverseName() == null ? (activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName()) : stmtTypeDrop.getDataverseName().getValue();
-            if (dataverseName == null) {
-                throw new AlgebricksException(" dataverse not specified ");
-            }
+            String dataverseName = getActiveDataverseName(stmtTypeDrop.getDataverseName());
             String typeName = stmtTypeDrop.getTypeName().getValue();
             Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
             if (dt == null) {
@@ -1078,7 +1052,7 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -1104,7 +1078,7 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -1117,11 +1091,7 @@
 
         try {
             CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
-            String dataverse = cfs.getSignature().getNamespace() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : cfs.getSignature().getNamespace();
-            if (dataverse == null) {
-                throw new AlgebricksException(" dataverse not specified ");
-            }
+            String dataverse = getActiveDataverseName(cfs.getSignature().getNamespace());
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
             if (dv == null) {
                 throw new AlgebricksException("There is no dataverse with this name " + dataverse + ".");
@@ -1134,7 +1104,7 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -1158,7 +1128,7 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -1174,8 +1144,7 @@
         List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
         try {
             LoadFromFileStatement loadStmt = (LoadFromFileStatement) stmt;
-            String dataverseName = loadStmt.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : loadStmt.getDataverseName().getValue();
+            String dataverseName = getActiveDataverseName(loadStmt.getDataverseName());
             CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(dataverseName, loadStmt
                     .getDatasetName().getValue(), loadStmt.getAdapter(), loadStmt.getProperties(),
                     loadStmt.dataIsAlreadySorted());
@@ -1207,7 +1176,7 @@
                 abort(e, e, mdTxnCtx);
             }
 
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseReadLatch();
         }
@@ -1223,8 +1192,7 @@
         try {
             metadataProvider.setWriteTransaction(true);
             WriteFromQueryResultStatement st1 = (WriteFromQueryResultStatement) stmt;
-            String dataverseName = st1.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : st1.getDataverseName().getValue();
+            String dataverseName = getActiveDataverseName(st1.getDataverseName());
             CompiledWriteFromQueryResultStatement clfrqs = new CompiledWriteFromQueryResultStatement(dataverseName, st1
                     .getDatasetName().getValue(), st1.getQuery(), st1.getVarCounter());
 
@@ -1238,7 +1206,7 @@
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
             }
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseReadLatch();
         }
@@ -1255,8 +1223,7 @@
         try {
             metadataProvider.setWriteTransaction(true);
             InsertStatement stmtInsert = (InsertStatement) stmt;
-            String dataverseName = stmtInsert.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : stmtInsert.getDataverseName().getValue();
+            String dataverseName = getActiveDataverseName(stmtInsert.getDataverseName());
             CompiledInsertStatement clfrqs = new CompiledInsertStatement(dataverseName, stmtInsert.getDatasetName()
                     .getValue(), stmtInsert.getQuery(), stmtInsert.getVarCounter());
             JobSpecification compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
@@ -1272,7 +1239,7 @@
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
             }
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseReadLatch();
         }
@@ -1289,8 +1256,7 @@
         try {
             metadataProvider.setWriteTransaction(true);
             DeleteStatement stmtDelete = (DeleteStatement) stmt;
-            String dataverseName = stmtDelete.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : stmtDelete.getDataverseName().getValue();
+            String dataverseName = getActiveDataverseName(stmtDelete.getDataverseName());
             CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(stmtDelete.getVariableExpr(), dataverseName,
                     stmtDelete.getDatasetName().getValue(), stmtDelete.getCondition(), stmtDelete.getVarCounter(),
                     metadataProvider);
@@ -1307,7 +1273,7 @@
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
             }
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseReadLatch();
         }
@@ -1340,8 +1306,7 @@
 
         try {
             BeginFeedStatement bfs = (BeginFeedStatement) stmt;
-            String dataverseName = bfs.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : bfs.getDataverseName().getValue();
+            String dataverseName = getActiveDataverseName(bfs.getDataverseName());
 
             CompiledBeginFeedStatement cbfs = new CompiledBeginFeedStatement(dataverseName, bfs.getDatasetName()
                     .getValue(), bfs.getQuery(), bfs.getVarCounter());
@@ -1374,7 +1339,7 @@
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
             }
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseReadLatch();
         }
@@ -1389,8 +1354,7 @@
 
         try {
             ControlFeedStatement cfs = (ControlFeedStatement) stmt;
-            String dataverseName = cfs.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                    : activeDefaultDataverse.getDataverseName() : cfs.getDatasetName().getValue();
+            String dataverseName = getActiveDataverseName(cfs.getDataverseName());
             CompiledControlFeedStatement clcfs = new CompiledControlFeedStatement(cfs.getOperationType(),
                     dataverseName, cfs.getDatasetName().getValue(), cfs.getAlterAdapterConfParams());
             JobSpecification jobSpec = FeedOperations.buildControlFeedJobSpec(clcfs, metadataProvider);
@@ -1404,7 +1368,7 @@
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
             }
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseReadLatch();
         }
@@ -1435,38 +1399,49 @@
                     handle.put(jobId.getId());
                     handle.put(metadataProvider.getResultSetId().getId());
                     response.put("handle", handle);
+                    out.print(response);
+                    out.flush();
                 } else {
+                    if (pdf == DisplayFormat.HTML) {
+                        out.println("<h4>Results:</h4>");
+                        out.println("<pre>");
+                    }
+
                     ByteBuffer buffer = ByteBuffer.allocate(ResultReader.FRAME_SIZE);
                     ResultReader resultReader = new ResultReader(hcc, hdc);
                     resultReader.open(jobId, metadataProvider.getResultSetId());
                     buffer.clear();
-                    JSONArray results = new JSONArray();
+
                     while (resultReader.read(buffer) > 0) {
-                        results.put(ResultUtils.getJSONFromBuffer(buffer, resultReader.getFrameTupleAccessor()));
+                        response.put("results",
+                                ResultUtils.getJSONFromBuffer(buffer, resultReader.getFrameTupleAccessor()));
                         buffer.clear();
+                        switch (pdf) {
+                            case HTML:
+                                ResultUtils.prettyPrintHTML(out, response);
+                                break;
+                            case TEXT:
+                            case JSON:
+                                out.print(response);
+                                break;
+                        }
+                        out.flush();
                     }
-                    response.put("results", results);
-                }
-                switch (pdf) {
-                    case HTML:
-                        out.println("<pre>");
-                        ResultUtils.prettyPrintHTML(out, response);
+                    if (pdf == DisplayFormat.HTML) {
                         out.println("</pre>");
-                        break;
-                    case TEXT:
-                    case JSON:
-                        out.print(response);
-                        break;
+                    }
+
                 }
                 hcc.waitForCompletion(jobId);
             }
 
             return queryResult;
         } catch (Exception e) {
+            e.printStackTrace();
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
             }
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseReadLatch();
         }
@@ -1496,7 +1471,7 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
-            throw new AlgebricksException(e);
+            throw e;
         } finally {
             releaseWriteLatch();
         }
@@ -1535,6 +1510,20 @@
         return format;
     }
 
+    private String getActiveDataverseName(String dataverse) throws AlgebricksException {
+        if (dataverse != null) {
+            return dataverse;
+        }
+        if (activeDefaultDataverse != null) {
+            return activeDefaultDataverse.getDataverseName();
+        }
+        throw new AlgebricksException("dataverse not specified");
+    }
+
+    private String getActiveDataverseName(Identifier dataverse) throws AlgebricksException {
+        return getActiveDataverseName(dataverse != null ? dataverse.getValue() : null);
+    }
+
     private void acquireWriteLatch() {
         MetadataManager.INSTANCE.acquireWriteLatch();
     }
@@ -1559,4 +1548,4 @@
             throw new IllegalStateException(rootE);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index 3600bea..fde3165 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -22,12 +22,13 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.api.common.Job;
-import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
-import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -40,10 +41,11 @@
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledDatasetDropStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -112,10 +114,10 @@
             throw new AlgebricksException("DROP DATASET: No metadata for dataset " + datasetName);
         }
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            return new JobSpecification();
+            return JobSpecificationUtils.createJobSpecification();
         }
 
-        JobSpecification specPrimary = new JobSpecification();
+        JobSpecification specPrimary = JobSpecificationUtils.createJobSpecification();
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(), datasetName,
@@ -123,12 +125,11 @@
         AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
-                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
                         storageProperties.getBloomFilterFalsePositiveRate()));
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
                 splitsAndConstraint.second);
@@ -152,7 +153,7 @@
             throw new AsterixException("Could not find dataset " + datasetName + " in datavetse " + dataverseName);
         }
         ARecordType itemType = (ARecordType) metadata.findType(dataverseName, dataset.getItemTypeName());
-        JobSpecification spec = new JobSpecification();
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
         IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
                 itemType, format.getBinaryComparatorFactoryProvider());
         ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
@@ -170,20 +171,19 @@
         AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
-                comparatorFactories, blooFilterKeyFields, true, storageProperties.getMemoryComponentPageSize(),
-                storageProperties.getMemoryComponentNumPages());
+                comparatorFactories, blooFilterKeyFields, true, fs, dataset.getDatasetId());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMBTreeResource);
 
         TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields,
-                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
-                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                        storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
+                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+                                .getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
                 splitsAndConstraint.second);
@@ -205,7 +205,7 @@
             throw new AsterixException("Cannot load data into dataset  (" + datasetName + ")" + "of type "
                     + dataset.getDatasetType());
         }
-        JobSpecification spec = new JobSpecification();
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
         ARecordType itemType = (ARecordType) MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName,
                 dataset.getItemTypeName()).getDatatype();
@@ -269,11 +269,11 @@
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields, fieldPermutation,
                 GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, numElementsHint, new LSMBTreeDataflowHelperFactory(
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
                         storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
                 splitsAndConstraint.second);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DataverseOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DataverseOperations.java
index 0654ff7..b71a44a 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DataverseOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DataverseOperations.java
@@ -11,7 +11,7 @@
 
 public class DataverseOperations {
     public static JobSpecification createDropDataverseJobSpec(Dataverse dataverse, AqlMetadataProvider metadata) {
-        JobSpecification jobSpec = new JobSpecification();
+        JobSpecification jobSpec = JobSpecificationUtils.createJobSpecification();
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForDataverse(dataverse.getDataverseName());
         FileRemoveOperatorDescriptor frod = new FileRemoveOperatorDescriptor(jobSpec, splitsAndConstraint.first);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
index f9bd2d5..27b32f8 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
@@ -93,7 +93,7 @@
             throw new AsterixException("Operation not support for dataset type  " + dataset.getDatasetType());
         }
 
-        JobSpecification spec = new JobSpecification();
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
         IOperatorDescriptor feedMessenger;
         AlgebricksPartitionConstraint messengerPc;
 
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
index 67bdabb..204d5b2 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
@@ -1,12 +1,14 @@
 package edu.uci.ics.asterix.file;
 
-import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
-import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -39,25 +41,23 @@
     }
 
     public static JobSpecification buildDropSecondaryIndexJobSpec(CompiledIndexDropStatement indexDropStmt,
-            AqlMetadataProvider metadataProvider) throws AlgebricksException, MetadataException {
+            AqlMetadataProvider metadataProvider, Dataset dataset) throws AlgebricksException, MetadataException {
         String dataverseName = indexDropStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
                 : indexDropStmt.getDataverseName();
         String datasetName = indexDropStmt.getDatasetName();
         String indexName = indexDropStmt.getIndexName();
-
-        JobSpecification spec = new JobSpecification();
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, indexName);
         AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
-                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
                         storageProperties.getBloomFilterFalsePositiveRate()));
         AlgebricksPartitionConstraintHelper
                 .setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/JobSpecificationUtils.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/JobSpecificationUtils.java
new file mode 100644
index 0000000..fa0a40f
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/JobSpecificationUtils.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.asterix.file;
+
+import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class JobSpecificationUtils {
+    public static JobSpecification createJobSpecification() {
+        JobSpecification spec = new JobSpecification();
+        AsterixCompilerProperties compilerProperties = AsterixAppContextInfo.getInstance().getCompilerProperties();
+        int frameSize = compilerProperties.getFrameSize();
+        spec.setFrameSize(frameSize);
+
+        return spec;
+    }
+}
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
index c3a2c01..d15d2bb 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -1,12 +1,13 @@
 package edu.uci.ics.asterix.file;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
-import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
@@ -34,25 +35,24 @@
 
     @Override
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = new JobSpecification();
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
         AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
-                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, false,
-                storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages());
+                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, true,
+                secondaryFileSplitProvider.getFileSplits(), dataset.getDatasetId());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMBTreeResource);
 
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
-                secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
                         storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
@@ -64,7 +64,7 @@
 
     @Override
     public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = new JobSpecification();
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
         // Create dummy key provider for feeding the primary index scan. 
         AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
@@ -89,12 +89,12 @@
         TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
                 spec,
                 numSecondaryKeys,
-                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
-                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                        storageProperties.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, storageProperties
+                                .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
index 253df4b..46f5b1a 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
@@ -19,11 +19,10 @@
 import java.io.IOException;
 import java.util.List;
 
-import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
-import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
@@ -37,9 +36,12 @@
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.asterix.runtime.evaluators.functions.AndDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.IsNullDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NotDescriptor;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -76,7 +78,8 @@
 import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 
 @SuppressWarnings("rawtypes")
-// TODO: We should eventually have a hierarchy of classes that can create all possible index job specs, 
+// TODO: We should eventually have a hierarchy of classes that can create all
+// possible index job specs,
 // not just for creation.
 public abstract class SecondaryIndexCreator {
     protected final PhysicalOptimizationConfig physOptConf;
@@ -278,12 +281,12 @@
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
                 primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
-                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
-                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                        storageProperties.getBloomFilterFalsePositiveRate()), false,
+                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+                                .getBloomFilterFalsePositiveRate()), false,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
@@ -336,16 +339,14 @@
         for (int i = 0; i < numSecondaryKeyFields + numPrimaryKeys; i++) {
             fieldPermutation[i] = i;
         }
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
-                        secondaryIndexName);
         TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                secondarySplitsAndConstraint.first, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
-                secondaryBloomFilterKeyFields, fieldPermutation, fillFactor, false, numElementsHint,
-                dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
+                AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, secondaryFileSplitProvider,
+                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+                fieldPermutation, fillFactor, false, numElementsHint, dataflowHelperFactory,
+                NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
-                secondarySplitsAndConstraint.second);
+                secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
     }
 
@@ -365,7 +366,8 @@
         }
         ICopyEvaluatorFactory selectCond = null;
         if (numSecondaryKeyFields > 1) {
-            // Create conjunctive condition where all secondary index keys must satisfy 'is not null'.
+            // Create conjunctive condition where all secondary index keys must
+            // satisfy 'is not null'.
             AndDescriptor andDesc = new AndDescriptor();
             selectCond = andDesc.createEvaluatorFactory(andArgsEvalFactories);
         } else {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
index 366e247..ae06966 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
@@ -2,19 +2,20 @@
 
 import java.util.List;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
-import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.asterix.runtime.formats.FormatUtils;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.LSMInvertedIndexLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -150,14 +151,12 @@
 
     @Override
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = new JobSpecification();
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
-        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMInvertedIndexLocalResourceMetadata(invListsTypeTraits,
-                primaryComparatorFactories, tokenTypeTraits, tokenComparatorFactories, tokenizerFactory,
-                storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                isPartitioned);
+                primaryComparatorFactories, tokenTypeTraits, tokenComparatorFactories, tokenizerFactory, isPartitioned,
+                secondaryFileSplitProvider.getFileSplits(), dataset.getDatasetId());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMInvertedIndexResource);
 
@@ -176,7 +175,7 @@
 
     @Override
     public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = new JobSpecification();
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
         // Create dummy key provider for feeding the primary index scan.
         AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
@@ -265,20 +264,18 @@
     private IIndexDataflowHelperFactory createDataflowHelperFactory() {
         AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         if (!isPartitioned) {
-            return new LSMInvertedIndexDataflowHelperFactory(
+            return new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                     storageProperties.getBloomFilterFalsePositiveRate());
         } else {
-            return new PartitionedLSMInvertedIndexDataflowHelperFactory(
+            return new PartitionedLSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
                     storageProperties.getBloomFilterFalsePositiveRate());
         }
     }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
index 89a59e8..d51a1bb 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
@@ -2,9 +2,10 @@
 
 import java.util.List;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
-import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
@@ -15,9 +16,9 @@
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -57,15 +58,15 @@
 
     @Override
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = new JobSpecification();
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
         AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMRTreeLocalResourceMetadata(
                 secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, primaryComparatorFactories,
                 valueProviderFactories, RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
-                        secondaryComparatorFactories.length), storageProperties.getMemoryComponentPageSize(),
-                storageProperties.getMemoryComponentNumPages());
+                        secondaryComparatorFactories.length), secondaryFileSplitProvider.getFileSplits(),
+                dataset.getDatasetId());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMRTreeResource);
 
@@ -73,13 +74,13 @@
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, null,
                 new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
-                        primaryComparatorFactories, AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                        primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
                                 keyType, secondaryComparatorFactories.length), storageProperties
-                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                        storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
+                                .getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
@@ -135,7 +136,7 @@
 
     @Override
     public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = new JobSpecification();
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
         // Create dummy key provider for feeding the primary index scan. 
         AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
@@ -159,13 +160,13 @@
                 spec,
                 numNestedSecondaryKeyFields,
                 new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
-                        primaryComparatorFactories, AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                        primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
                                 keyType, secondaryComparatorFactories.length), storageProperties
-                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                        storageProperties.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+                                .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 01cbca0..9b83f00 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -13,12 +13,13 @@
 import edu.uci.ics.asterix.api.http.servlet.QueryResultAPIServlet;
 import edu.uci.ics.asterix.api.http.servlet.QueryStatusAPIServlet;
 import edu.uci.ics.asterix.api.http.servlet.UpdateAPIServlet;
-import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
+import edu.uci.ics.asterix.common.api.AsterixThreadFactory;
 import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
 import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
 import edu.uci.ics.asterix.metadata.bootstrap.AsterixStateProxy;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
 import edu.uci.ics.hyracks.api.application.ICCApplicationEntryPoint;
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
@@ -37,12 +38,12 @@
     @Override
     public void start(ICCApplicationContext ccAppCtx, String[] args) throws Exception {
         this.appCtx = ccAppCtx;
+
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting Asterix cluster controller");
         }
-
+        appCtx.setThreadFactory(AsterixThreadFactory.INSTANCE);
         AsterixAppContextInfo.initialize(appCtx);
-
         proxy = AsterixStateProxy.registerRemoteObject();
         appCtx.setDistributedState(proxy);
 
@@ -54,6 +55,7 @@
         webServer.start();
         setupJSONAPIServer(externalProperties);
         jsonAPIServer.start();
+        ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
     }
 
     @Override
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
new file mode 100644
index 0000000..e8a1ad2
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.hyracks.bootstrap;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.hyracks.api.application.IClusterLifecycleListener;
+
+public class ClusterLifecycleListener implements IClusterLifecycleListener {
+
+    public static ClusterLifecycleListener INSTANCE = new ClusterLifecycleListener();
+
+    private ClusterLifecycleListener() {
+    }
+
+    private static final Logger LOGGER = Logger.getLogger(ClusterLifecycleListener.class.getName());
+
+    @Override
+    public void notifyNodeJoin(String nodeId, Map<String, String> ncConfiguration) {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("NC: " + nodeId + " joined");
+        }
+        AsterixClusterProperties.INSTANCE.addNCConfiguration(nodeId, ncConfiguration);
+    }
+
+    public void notifyNodeFailure(Set<String> deadNodeIds) {
+        for (String deadNode : deadNodeIds) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("NC: " + deadNode + " left");
+            }
+            AsterixClusterProperties.INSTANCE.removeNCConfiguration(deadNode);
+        }
+
+    }
+
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index 7dad984..32bfeb5 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -1,28 +1,35 @@
 package edu.uci.ics.asterix.hyracks.bootstrap;
 
+import java.io.File;
 import java.rmi.RemoteException;
 import java.rmi.server.UnicastRemoteObject;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.asterix.api.common.AsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.api.AsterixThreadFactory;
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
-import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.SystemState;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataNode;
 import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
 import edu.uci.ics.asterix.metadata.api.IMetadataNode;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataBootstrap;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager;
-import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager.SystemState;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.application.INCApplicationEntryPoint;
+import edu.uci.ics.hyracks.api.lifecycle.LifeCycleComponentManager;
 
 public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
     private static final Logger LOGGER = Logger.getLogger(NCApplicationEntryPoint.class.getName());
 
     private INCApplicationContext ncApplicationContext = null;
-    private AsterixAppRuntimeContext runtimeContext;
+    private IAsterixAppRuntimeContext runtimeContext;
     private String nodeId;
     private boolean isMetadataNode = false;
     private boolean stopInitiated = false;
@@ -30,17 +37,19 @@
 
     @Override
     public void start(INCApplicationContext ncAppCtx, String[] args) throws Exception {
+        ncAppCtx.setThreadFactory(AsterixThreadFactory.INSTANCE);
         ncApplicationContext = ncAppCtx;
         nodeId = ncApplicationContext.getNodeId();
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Starting Asterix node controller: " + nodeId);
+            LOGGER.info("Starting Asterix node controller  TAKE NOTE: " + nodeId);
         }
+        JVMShutdownHook sHook = new JVMShutdownHook(this);
+        Runtime.getRuntime().addShutdownHook(sHook);
 
+     
         runtimeContext = new AsterixAppRuntimeContext(ncApplicationContext);
         runtimeContext.initialize();
         ncApplicationContext.setApplicationObject(runtimeContext);
-        JVMShutdownHook sHook = new JVMShutdownHook(this);
-        Runtime.getRuntime().addShutdownHook(sHook);
 
         // #. recover if the system is corrupted by checking system state.
         IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
@@ -76,6 +85,8 @@
             if (isMetadataNode) {
                 MetadataBootstrap.stopUniverse();
             }
+
+            LifeCycleComponentManager.INSTANCE.stopAll(false);
             runtimeContext.deinitialize();
         } else {
             if (LOGGER.isLoggable(Level.INFO)) {
@@ -87,7 +98,8 @@
     @Override
     public void notifyStartupComplete() throws Exception {
         IAsterixStateProxy proxy = (IAsterixStateProxy) ncApplicationContext.getDistributedState();
-        AsterixMetadataProperties metadataProperties = runtimeContext.getMetadataProperties();
+        AsterixMetadataProperties metadataProperties = ((IAsterixPropertiesProvider) runtimeContext)
+                .getMetadataProperties();
 
         if (systemState == SystemState.NEW_UNIVERSE) {
             if (LOGGER.isLoggable(Level.INFO)) {
@@ -111,11 +123,29 @@
             }
             MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
             MetadataManager.INSTANCE.init();
-            MetadataBootstrap.startUniverse(runtimeContext, ncApplicationContext,
+            MetadataBootstrap.startUniverse(((IAsterixPropertiesProvider) runtimeContext), ncApplicationContext,
                     systemState == SystemState.NEW_UNIVERSE);
             MetadataBootstrap.startDDLRecovery();
         }
 
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Starting lifecycle components");
+        }
+        
+        Map<String, String> lifecycleMgmtConfiguration = new HashMap<String, String>();
+        String key = LifeCycleComponentManager.Config.DUMP_PATH_KEY;
+        String value = metadataProperties.getCoredumpPath(nodeId);
+        lifecycleMgmtConfiguration.put(key, value);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Coredump directory for NC is: " + value);
+        }
+        LifeCycleComponentManager.INSTANCE.configure(lifecycleMgmtConfiguration);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Configured:" + LifeCycleComponentManager.INSTANCE);
+        }
+        
+        LifeCycleComponentManager.INSTANCE.startAll();
+
         IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
         recoveryMgr.checkpoint(true);
 
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java
index dec3128..012b7b2 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java
@@ -14,14 +14,22 @@
  */
 package edu.uci.ics.asterix.result;
 
+import java.io.BufferedReader;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.io.PrintWriter;
+import java.io.StringWriter;
 import java.nio.ByteBuffer;
 
 import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import com.sun.el.parser.ParseException;
+
+import edu.uci.ics.asterix.api.http.servlet.APIServlet;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
@@ -51,13 +59,18 @@
         return resultRecords;
     }
 
-    public static JSONObject getErrorResponse(int errorCode, String errorMessage) {
+    public static JSONObject getErrorResponse(int errorCode, String errorMessage, String errorSummary,
+            String errorStackTrace) {
         JSONObject errorResp = new JSONObject();
         JSONArray errorArray = new JSONArray();
         errorArray.put(errorCode);
         errorArray.put(errorMessage);
         try {
             errorResp.put("error-code", errorArray);
+            if (!errorSummary.equals(""))
+                errorResp.put("summary", errorSummary);
+            if (!errorStackTrace.equals(""))
+                errorResp.put("stacktrace", errorStackTrace);
         } catch (JSONException e) {
             // TODO(madhusudancs): Figure out what to do when JSONException occurs while building the results.
         }
@@ -65,18 +78,159 @@
     }
 
     public static void prettyPrintHTML(PrintWriter out, JSONObject jsonResultObj) {
-        JSONArray resultsWrapper;
-        JSONArray resultsArray;
         try {
-            resultsWrapper = jsonResultObj.getJSONArray("results");
-            for (int i = 0; i < resultsWrapper.length(); i++) {
-                resultsArray = resultsWrapper.getJSONArray(i);
-                for (int j = 0; j < resultsArray.length(); j++) {
-                    out.print(resultsArray.getString(j));
-                }
+            JSONArray resultsArray = jsonResultObj.getJSONArray("results");
+            for (int i = 0; i < resultsArray.length(); i++) {
+                out.print(resultsArray.getString(i));
             }
         } catch (JSONException e) {
             // TODO(madhusudancs): Figure out what to do when JSONException occurs while building the results.
         }
     }
+
+    public static void webUIErrorHandler(PrintWriter out, Exception e) {
+        String errorTemplate = readTemplateFile("/webui/errortemplate.html", "%s\n%s\n%s");
+
+        String errorOutput = String.format(errorTemplate, extractErrorMessage(e), extractErrorSummary(e),
+                extractFullStackTrace(e));
+        out.println(errorOutput);
+    }
+
+    public static void webUIParseExceptionHandler(PrintWriter out, Throwable e, String query) {
+        String errorTemplate = readTemplateFile("/webui/errortemplate_message.html", "<pre class=\"error\">%s\n</pre>");
+
+        String errorOutput = String.format(errorTemplate, buildParseExceptionMessage(e, query));
+        out.println(errorOutput);
+    }
+
+    public static void apiErrorHandler(PrintWriter out, Exception e) {
+        int errorCode = 99;
+        if (e instanceof ParseException) {
+            errorCode = 2;
+        } else if (e instanceof AlgebricksException) {
+            errorCode = 3;
+        } else if (e instanceof HyracksDataException) {
+            errorCode = 4;
+        }
+
+        JSONObject errorResp = ResultUtils.getErrorResponse(errorCode, extractErrorMessage(e), extractErrorSummary(e),
+                extractFullStackTrace(e));
+        out.write(errorResp.toString());
+    }
+
+    public static String buildParseExceptionMessage(Throwable e, String query) {
+        StringBuilder errorMessage = new StringBuilder();
+        String message = e.getMessage();
+        message = message.replace("<", "&lt");
+        message = message.replace(">", "&gt");
+        errorMessage.append("SyntaxError:" + message + "\n");
+        int pos = message.indexOf("line");
+        if (pos > 0) {
+            int columnPos = message.indexOf(",", pos + 1 + "line".length());
+            int lineNo = Integer.parseInt(message.substring(pos + "line".length() + 1, columnPos));
+            String[] lines = query.split("\n");
+            if (lineNo >= lines.length) {
+                errorMessage.append("===> &ltBLANK LINE&gt \n");
+            } else {
+                String line = lines[lineNo - 1];
+                errorMessage.append("==> " + line);
+            }
+        }
+        return errorMessage.toString();
+    }
+
+    private static Throwable getRootCause(Throwable cause) {
+        Throwable nextCause = cause.getCause();
+        while (nextCause != null) {
+            cause = nextCause;
+            nextCause = cause.getCause();
+        }
+        return cause;
+    }
+
+    /**
+     * Extract the message in the root cause of the stack trace:
+     * 
+     * @param e
+     * @return error message string.
+     */
+    private static String extractErrorMessage(Throwable e) {
+        Throwable cause = getRootCause(e);
+
+        String exceptionClassName = "";
+        String[] messageSplits = cause.toString().split(":");
+        if (messageSplits.length > 1) {
+            String fullyQualifiedExceptionClassName = messageSplits[0];
+            System.out.println(fullyQualifiedExceptionClassName);
+            String[] hierarchySplits = fullyQualifiedExceptionClassName.split("\\.");
+            if (hierarchySplits.length > 0) {
+                exceptionClassName = hierarchySplits[hierarchySplits.length - 1];
+            }
+        }
+        return cause.getLocalizedMessage() + " [" + exceptionClassName + "]";
+    }
+
+    /**
+     * Extract the meaningful part of a stack trace:
+     * a. the causes in the stack trace hierarchy
+     * b. the top exception for each cause
+     * 
+     * @param e
+     * @return the contacted message containing a and b.
+     */
+    private static String extractErrorSummary(Throwable e) {
+        StringBuilder errorMessageBuilder = new StringBuilder();
+        Throwable cause = e;
+        errorMessageBuilder.append(cause.getLocalizedMessage());
+        while (cause != null) {
+            StackTraceElement[] stackTraceElements = cause.getStackTrace();
+            errorMessageBuilder.append(stackTraceElements.length > 0 ? "\n caused by: " + stackTraceElements[0] : "");
+            cause = cause.getCause();
+        }
+        return errorMessageBuilder.toString();
+    }
+
+    /**
+     * Extract the full stack trace:
+     * 
+     * @param e
+     * @return the string containing the full stack trace of the error.
+     */
+    private static String extractFullStackTrace(Throwable e) {
+        StringWriter stringWriter = new StringWriter();
+        PrintWriter printWriter = new PrintWriter(stringWriter);
+        e.printStackTrace(printWriter);
+        return stringWriter.toString();
+    }
+
+    /**
+     * Read the template file which is stored as a resource and return its content. If the file does not exist or is
+     * not readable return the default template string.
+     *
+     * @param path
+     *            The path to the resource template file
+     * @param defaultTemplate
+     *            The default template string if the template file does not exist or is not readable
+     * @return The template string to be used to render the output.
+     */
+    private static String readTemplateFile(String path, String defaultTemplate) {
+        String errorTemplate = defaultTemplate;
+        try {
+            String resourcePath = "/webui/errortemplate_message.html";
+            InputStream is = APIServlet.class.getResourceAsStream(resourcePath);
+            InputStreamReader isr = new InputStreamReader(is);
+            StringBuilder sb = new StringBuilder();
+            BufferedReader br = new BufferedReader(isr);
+            String line = br.readLine();
+
+            while (line != null) {
+                sb.append(line);
+                line = br.readLine();
+            }
+            errorTemplate = sb.toString();
+        } catch (IOException ioe) {
+            // If there is an IOException reading the error template html file, default value of error template is used.
+        }
+        return errorTemplate;
+    }
 }
diff --git a/asterix-app/src/main/resources/asterix-build-configuration.xml b/asterix-app/src/main/resources/asterix-build-configuration.xml
index 3e1b4b2..e85e568 100644
--- a/asterix-app/src/main/resources/asterix-build-configuration.xml
+++ b/asterix-app/src/main/resources/asterix-build-configuration.xml
@@ -13,4 +13,16 @@
      <value>WARNING</value>
      <description>Log level for running tests/build</description>
   </property>
+  <property>
+		<name>storage.memorycomponent.numpages</name>
+		<value>8</value>
+		<description>The number of pages to allocate for a memory component.
+			(Default = 8)
+		</description>
+	</property>
+  <property>
+     <name>txn.log.groupcommitinterval</name>
+     <value>1</value>
+     <description>The group commit wait time in milliseconds.</description>
+  </property>
 </asterixConfiguration>
diff --git a/asterix-app/src/main/resources/log.properties b/asterix-app/src/main/resources/log.properties
deleted file mode 100644
index ee8040a..0000000
--- a/asterix-app/src/main/resources/log.properties
+++ /dev/null
@@ -1 +0,0 @@
-group_commit_wait_period=1
\ No newline at end of file
diff --git a/asterix-app/src/main/resources/webui/errortemplate.html b/asterix-app/src/main/resources/webui/errortemplate.html
new file mode 100644
index 0000000..8c39e03
--- /dev/null
+++ b/asterix-app/src/main/resources/webui/errortemplate.html
@@ -0,0 +1,38 @@
+<div class="accordion" id="errorblock">
+  <div class="accordion-group">
+    <div class="accordion-heading">
+      <a class="accordion-toggle" data-toggle="collapse" data-parent="#errorblock" href="#levelOne">
+        Message
+      </a>
+    </div>
+    <div id="levelOne" class="accordion-body collapse in">
+      <div class="accordion-inner">
+        <pre class="error">%s</pre>
+      </div>
+    </div>
+  </div>
+  <div class="accordion-group">
+    <div class="accordion-heading">
+      <a class="accordion-toggle" data-toggle="collapse" data-parent="#errorblock" href="#levelTwo">
+        Summary
+      </a>
+    </div>
+    <div id="levelTwo" class="accordion-body collapse">
+      <div class="accordion-inner">
+        <pre class="error">%s</pre>
+      </div>
+    </div>
+  </div>
+  <div class="accordion-group">
+    <div class="accordion-heading">
+      <a class="accordion-toggle" data-toggle="collapse" data-parent="#errorblock" href="#levelThree">
+        Full Stack Trace
+      </a>
+    </div>
+    <div id="levelThree" class="accordion-body collapse">
+      <div class="accordion-inner">
+        <pre class="error">%s</pre>
+      </div>
+    </div>
+  </div>
+</div>
diff --git a/asterix-app/src/main/resources/webui/errortemplate_message.html b/asterix-app/src/main/resources/webui/errortemplate_message.html
new file mode 100644
index 0000000..ea95ccf
--- /dev/null
+++ b/asterix-app/src/main/resources/webui/errortemplate_message.html
@@ -0,0 +1,14 @@
+<div class="accordion" id="errorblock">
+  <div class="accordion-group">
+    <div class="accordion-heading">
+      <a class="accordion-toggle" data-toggle="collapse" data-parent="#errorblock" href="#levelOne">
+        Message
+      </a>
+    </div>
+    <div id="levelOne" class="accordion-body collapse in">
+      <div class="accordion-inner">
+        <pre class="error">%s</pre>
+      </div>
+    </div>
+  </div>
+</div>
diff --git a/asterix-app/src/main/resources/webui/querytemplate.html b/asterix-app/src/main/resources/webui/querytemplate.html
index 95323bb..f1c2e7e 100644
--- a/asterix-app/src/main/resources/webui/querytemplate.html
+++ b/asterix-app/src/main/resources/webui/querytemplate.html
@@ -14,23 +14,127 @@
 <link href="/webui/static/css/style.css" rel="stylesheet" type="text/css" />
 
 <script type="text/javascript">
-$(document).ready(function(){
-   $("form#queryform").submit(function() {
-     $('#output-message').html("");
-     $.post("/", $("form#queryform").serialize(), function(data) {
-       $('#output-message').html(data);
-     });
-     return false;
-   });
+$(document).ready(function() {
+
+    var optionButtonSize = $('#checkboxes-on').width();
+    $('#clear-query-button, #run-btn').width(optionButtonSize);
+
+    $('#checkboxes-on').click(function() {
+        /* Displays a checkmark to indicate selection/clearing */
+        if ($('#opts').is(":visible")) {
+            $('#opts').hide();
+            $('#queryform :input').prop('checked', false);
+        } else {
+            $('#opts').show();    
+            $('#queryform :input').prop('checked', true);
+        }    
+        return false;
+    });
+
+    $('#clear-query-button').click(function() {
+        $("#qry").val('');
+        return false;
+    });
+
+    $('form#queryform :input').click( function () {
+        /* Hides selection check on uncheck, shows when all 5 selected */
+        if ($(this).val()) {
+            if ($(this).is(':checked') && $('input[type=checkbox]').filter(':checked').length == 5) {
+                $('#opts').show();
+            } else {
+                $('#opts').hide();
+            }
+        }
+    });
+
+    $("form#queryform").submit(function() {
+        $('#output-message').html("");
+        $.post("/", $("form#queryform").serialize(), function(data) {
+
+            var resSet = 0;
+            var resPattern = /<h4>Results:<\/h4>/g;
+            var durPattern = /<PRE>Duration/g;
+            var errorPattern = /<div class="accordion" id="errorblock">/g;
+            var resultCount = data.match(resPattern);
+
+            if (!resPattern.test(data)) {
+                if(errorPattern.test(data)) {
+                  $('#output-heading').html('Error');
+                  $('#output-heading').addClass('error');
+                } else {
+                  $('#output-heading').html('Output');
+                  $('#output-heading').removeClass('error');
+                }
+                $('#output-message').html(data);
+            } else {
+                $('#output-heading').html('Output');
+                $('#output-heading').removeClass('error');
+                if (resultCount.length <= 1) {
+                    $('#output-message').html(data);
+                } else {
+                    var splitData = data.split('<PRE>Duration');
+                    var results = splitData[0].split('<h4>');
+                    var components = results.slice(1, results.length);
+                    var sections = components.length / resultCount.length;
+
+                    for (resSet = 0; resSet < resultCount.length; resSet++) {
+
+                        $('#output-message').append('<h4>' + components[(resSet+1)*sections - 1]);
+
+                        if (sections > 1) {
+                            var resNum = resSet + 1;
+                            $('<button/>')
+                                .attr("class", "btn")
+                                .attr("data-toggle", "collapse")
+                                .attr("data-target", "#collapse" + resSet)
+                                .css("margin-bottom", "1em")
+                                .html('Result Plan #' + resNum + '<i id="ibtn' + resSet + '" class="icon-plus extarget"></i>')
+                                .appendTo('#output-message');
+
+                            $('<div/>')
+                                .attr("id", "collapse" + resSet)
+                                .attr("class", "collapse in")
+                                .appendTo('#output-message');
+
+                            for (var c = 0; c < sections - 1; c++) {
+                                var pos = resSet*sections + c;
+                                $('#collapse' + resSet).append('<h4>' + components[pos]);
+                            }
+
+                            /* Placeholder for future on show/hide result plan behavior
+                            $('#collapse' + resSet).on('show', function() {
+                            }).on('hide', function() {
+                            });
+                            */
+
+                            $('#output-message').append("<hr/>");
+                        }
+
+                    }
+                    $('#output-message').append('<PRE>Duration' + splitData[1]);
+                }
+
+            }
+
+            var contentString = data.toString();
+            if (contentString.indexOf("<PRE>Duration") !== -1) {
+                $('<div/>')
+                    .addClass("alert alert-success")
+                    .html("Success: Query Complete")
+                    .appendTo('#output-message');
+            }
+        });
+        return false;
+    });
 });
 </script>
 
 <meta charset=utf-8 />
-<title>ASTERIX Demo</title>
+<title>AsterixDB Web Interface</title>
 </head>
 
 <body>
-  <div class="navbar navbar-inverse navbar-fixed-top">
+  <div class="navbar navbar-fixed-top">
     <div class="navbar-inner">
       <div class="container">
         <a class="btn btn-navbar" data-toggle="collapse" data-target=".nav-collapse">
@@ -38,12 +142,18 @@
           <span class="icon-bar"></span>
           <span class="icon-bar"></span>
         </a>
-        <a class="brand" href="#">ASTERIX</a>
+
+        <!-- Temporary logo placeholder -->
+        <a class="brand" href="#"><img src="/webui/static/img/finalasterixlogo.png"></a>
+
         <div class="nav-collapse collapse">
           <ul class="nav">
-            <li><a href="http://code.google.com/p/asterixdb/source/browse/">Open source</a></li>
-            <li><a href="http://code.google.com/p/asterixdb/issues/list">File issues</a></li>
-            <li><a href="https://groups.google.com/forum/?fromgroups#!forum/asterixdb-userst">Contact</a></li>
+            <li><a href="http://code.google.com/p/asterixdb/" target="_blank">
+                    Open source<img class="extarget" src="/webui/static/img/targetlink.png"/></a></li>
+            <li><a href="http://code.google.com/p/asterixdb/issues/list" target="_blank">
+                    File issues<img class="extarget" src="/webui/static/img/targetlink.png"/></a></li>
+            <li><a href="https://groups.google.com/forum/?fromgroups#!forum/asterixdb-users" target="_blank">
+                    Contact<img class="extarget" src="/webui/static/img/targetlink.png"/></a></li>
           </ul>
         </div><!--/.nav-collapse -->
       </div>
@@ -53,42 +163,48 @@
   <div class="content">
     <div class="container">
       <div class="row-fluid">
-        <div class="span6">
-          <form id="queryform" class="form-horizontal" method="post">
-            <div>
-              <label class="query">Query</label>
-              <textarea rows="10" name="query" class="query" value="%s" placeholder="Type your AQL query ..."></textarea>
-            </div>
-            <div>
-              <label class="checkbox"><input type="checkbox" checked="checked" name="print-expr-tree" value="true" /> Print parsed expressions</label>
-              <label class="checkbox"><input type="checkbox" checked="checked" name="print-rewritten-expr-tree" value="true" /> Print rewritten expressions</label>
-            </div>
-            <div>
-              <label class="checkbox"><input type="checkbox" checked="checked" name="print-logical-plan" value="true" /> Print logical plan</label>
-              <label class="checkbox"><input type="checkbox" checked="checked" name="print-optimized-logical-plan" value="true" /> Print optimized logical plan</label>
-            </div>
-            <div>
-              <label class="checkbox"><input type="checkbox" checked="checked" name="print-job" value="true" /> Print hyracks job</label>
-            </div>
-            <button type="submit" class="btn btn-danger">Execute</button>
-          </form>
-        </div>
 
         <div class="span6">
+
+          <form id="queryform" class="form-horizontal" method="post">
+            <div style="margin-bottom: 1em;">
+              <label class="query">Query</label>
+              <textarea rows="10" id="qry" name="query" class="query" value="%s" placeholder="Type your AQL query ..."></textarea>
+            </div>
+            
+          <div class="btn-group">
+            <button id="checkboxes-on" class="btn">
+                <i id="opts" class="icon-ok" style="display:none;"></i>Select Options</button>
+            <button id="clear-query-button" class="btn">Clear Query</button>
+            <!-- <button id="checkboxes-off" class="btn">Clear All Options</button> -->
+            <button type="submit" id="run-btn" class="btn btn-custom-darken">Run</button>
+          </div>
+
+            <div>
+              <label class="checkbox optlabel"><input type="checkbox" name="print-expr-tree" value="true" /> Print parsed expressions</label>
+              <label class="checkbox optlabel"><input type="checkbox" name="print-rewritten-expr-tree" value="true" /> Print rewritten expressions</label>
+              <label class="checkbox optlabel"><input type="checkbox" name="print-logical-plan" value="true" /> Print logical plan</label>
+              <label class="checkbox optlabel"><input type="checkbox" name="print-optimized-logical-plan" value="true" /> Print optimized logical plan</label>
+              <label class="checkbox optlabel"><input type="checkbox" name="print-job" value="true" /> Print Hyracks job</label>
+            </div>
+          </form>
+       </div>
+
+       <div class="span6">
          <div class="output">
-           <label class="heading">Output</label>
+           <label id="output-heading" class="heading">Output</label>
            <div id="output-message" class="message">
            </div>
          </div>
        </div>
+
       </div>
     </div>
-  </div>
+</div>
   <div class="footer">
     <section class="line"><hr></section>
     <section class="content">
       <section class="left">
-        Developed by ASTERIX group
       </section>
       <section class="right">
         &copy; Copyright 2013 University of California, Irvine
diff --git a/asterix-app/src/main/resources/webui/static/css/bootstrap.min.css b/asterix-app/src/main/resources/webui/static/css/bootstrap.min.css
index c10c7f4..518ad8e 100644
--- a/asterix-app/src/main/resources/webui/static/css/bootstrap.min.css
+++ b/asterix-app/src/main/resources/webui/static/css/bootstrap.min.css
@@ -1,9 +1,874 @@
 /*!
- * Bootstrap v2.3.1
+ * Bootstrap v2.3.2
  *
  * Copyright 2012 Twitter, Inc
  * Licensed under the Apache License v2.0
  * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Designed and built with all the love in the world @twitter by @mdo and @fat.
- */.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{width:auto\9;height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img,.google-maps img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,html input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}label,select,button,input[type="button"],input[type="reset"],input[type="submit"],input[type="radio"],input[type="checkbox"]{cursor:pointer}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}@media print{*{color:#000!important;text-shadow:none!important;background:transparent!important;box-shadow:none!important}a,a:visited{text-decoration:underline}a[href]:after{content:" (" attr(href) ")"}abbr[title]:after{content:" (" attr(title) ")"}.ir a:after,a[href^="javascript:"]:after,a[href^="#"]:after{content:""}pre,blockquote{border:1px solid #999;page-break-inside:avoid}thead{display:table-header-group}tr,img{page-break-inside:avoid}img{max-width:100%!important}@page{margin:.5cm}p,h2,h3{orphans:3;widows:3}h2,h3{page-break-after:avoid}}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover,a:focus{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;min-height:1px;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.127659574468085%}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:21px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}a.muted:hover,a.muted:focus{color:#808080}.text-warning{color:#c09853}a.text-warning:hover,a.text-warning:focus{color:#a47e3c}.text-error{color:#b94a48}a.text-error:hover,a.text-error:focus{color:#953b39}.text-info{color:#3a87ad}a.text-info:hover,a.text-info:focus{color:#2d6987}.text-success{color:#468847}a.text-success:hover,a.text-success:focus{color:#356635}.text-left{text-align:left}.text-right{text-align:right}.text-center{text-align:center}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:20px;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1,h2,h3{line-height:40px}h1{font-size:38.5px}h2{font-size:31.5px}h3{font-size:24.5px}h4{font-size:17.5px}h5{font-size:14px}h6{font-size:11.9px}h1 small{font-size:24.5px}h2 small{font-size:17.5px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}ul.inline,ol.inline{margin-left:0;list-style:none}ul.inline>li,ol.inline>li{display:inline-block;*display:inline;padding-right:5px;padding-left:5px;*zoom:1}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal{*zoom:1}.dl-horizontal:before,.dl-horizontal:after{display:table;line-height:0;content:""}.dl-horizontal:after{clear:both}.dl-horizontal dt{float:left;width:160px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:180px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title],abbr[data-original-title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:17.5px;font-weight:300;line-height:1.25}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;white-space:nowrap;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;white-space:pre;white-space:pre-wrap;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:10px;font-size:14px;line-height:20px;color:#555;vertical-align:middle;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}input,textarea,.uneditable-input{width:206px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #ccc}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:20px;padding-left:20px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-20px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"],.row-fluid .controls-row [class*="span"]{float:left}.controls-row .checkbox[class*="span"],.controls-row .radio[class*="span"]{padding-top:5px}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning .control-label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853}.control-group.warning input,.control-group.warning select,.control-group.warning textarea{border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error .control-label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48}.control-group.error input,.control-group.error select,.control-group.error textarea{border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success .control-label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847}.control-group.success input,.control-group.success select,.control-group.success textarea{border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}.control-group.info .control-label,.control-group.info .help-block,.control-group.info .help-inline{color:#3a87ad}.control-group.info .checkbox,.control-group.info .radio,.control-group.info input,.control-group.info select,.control-group.info textarea{color:#3a87ad}.control-group.info input,.control-group.info select,.control-group.info textarea{border-color:#3a87ad;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.info input:focus,.control-group.info select:focus,.control-group.info textarea:focus{border-color:#2d6987;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3}.control-group.info .input-prepend .add-on,.control-group.info .input-append .add-on{color:#3a87ad;background-color:#d9edf7;border-color:#3a87ad}input:focus:invalid,textarea:focus:invalid,select:focus:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:invalid:focus,textarea:focus:invalid:focus,select:focus:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{display:inline-block;margin-bottom:10px;font-size:0;white-space:nowrap;vertical-align:middle}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input,.input-append .dropdown-menu,.input-prepend .dropdown-menu,.input-append .popover,.input-prepend .popover{font-size:14px}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;vertical-align:top;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn,.input-append .btn-group>.dropdown-toggle,.input-prepend .btn-group>.dropdown-toggle{vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-append input+.btn-group .btn:last-child,.input-append select+.btn-group .btn:last-child,.input-append .uneditable-input+.btn-group .btn:last-child{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-append .add-on,.input-append .btn,.input-append .btn-group{margin-left:-1px}.input-append .add-on:last-child,.input-append .btn:last-child,.input-append .btn-group:last-child>.dropdown-toggle{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append input+.btn-group .btn,.input-prepend.input-append select+.btn-group .btn,.input-prepend.input-append .uneditable-input+.btn-group .btn{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append .btn-group:first-child{margin-left:0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:160px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:180px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:180px}.form-horizontal .help-block{margin-bottom:0}.form-horizontal input+.help-block,.form-horizontal select+.help-block,.form-horizontal textarea+.help-block,.form-horizontal .uneditable-input+.help-block,.form-horizontal .input-prepend+.help-block,.form-horizontal .input-append+.help-block{margin-top:10px}.form-horizontal .form-actions{padding-left:180px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table .table{background-color:#fff}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child>th:first-child,.table-bordered tbody:first-child tr:first-child>td:first-child,.table-bordered tbody:first-child tr:first-child>th:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child>th:last-child,.table-bordered tbody:first-child tr:first-child>td:last-child,.table-bordered tbody:first-child tr:first-child>th:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child>th:first-child,.table-bordered tbody:last-child tr:last-child>td:first-child,.table-bordered tbody:last-child tr:last-child>th:first-child,.table-bordered tfoot:last-child tr:last-child>td:first-child,.table-bordered tfoot:last-child tr:last-child>th:first-child{-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child>th:last-child,.table-bordered tbody:last-child tr:last-child>td:last-child,.table-bordered tbody:last-child tr:last-child>th:last-child,.table-bordered tfoot:last-child tr:last-child>td:last-child,.table-bordered tfoot:last-child tr:last-child>th:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered tfoot+tbody:last-child tr:last-child td:first-child{-webkit-border-bottom-left-radius:0;border-bottom-left-radius:0;-moz-border-radius-bottomleft:0}.table-bordered tfoot+tbody:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:0;border-bottom-right-radius:0;-moz-border-radius-bottomright:0}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-striped tbody>tr:nth-child(odd)>td,.table-striped tbody>tr:nth-child(odd)>th{background-color:#f9f9f9}.table-hover tbody tr:hover>td,.table-hover tbody tr:hover>th{background-color:#f5f5f5}table td[class*="span"],table th[class*="span"],.row-fluid table td[class*="span"],.row-fluid table th[class*="span"]{display:table-cell;float:none;margin-left:0}.table td.span1,.table th.span1{float:none;width:44px;margin-left:0}.table td.span2,.table th.span2{float:none;width:124px;margin-left:0}.table td.span3,.table th.span3{float:none;width:204px;margin-left:0}.table td.span4,.table th.span4{float:none;width:284px;margin-left:0}.table td.span5,.table th.span5{float:none;width:364px;margin-left:0}.table td.span6,.table th.span6{float:none;width:444px;margin-left:0}.table td.span7,.table th.span7{float:none;width:524px;margin-left:0}.table td.span8,.table th.span8{float:none;width:604px;margin-left:0}.table td.span9,.table th.span9{float:none;width:684px;margin-left:0}.table td.span10,.table th.span10{float:none;width:764px;margin-left:0}.table td.span11,.table th.span11{float:none;width:844px;margin-left:0}.table td.span12,.table th.span12{float:none;width:924px;margin-left:0}.table tbody tr.success>td{background-color:#dff0d8}.table tbody tr.error>td{background-color:#f2dede}.table tbody tr.warning>td{background-color:#fcf8e3}.table tbody tr.info>td{background-color:#d9edf7}.table-hover tbody tr.success:hover>td{background-color:#d0e9c6}.table-hover tbody tr.error:hover>td{background-color:#ebcccc}.table-hover tbody tr.warning:hover>td{background-color:#faf2cc}.table-hover tbody tr.info:hover>td{background-color:#c4e3f3}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav-pills>.active>a>[class^="icon-"],.nav-pills>.active>a>[class*=" icon-"],.nav-list>.active>a>[class^="icon-"],.nav-list>.active>a>[class*=" icon-"],.navbar-inverse .nav>.active>a>[class^="icon-"],.navbar-inverse .nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:focus>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>li>a:focus>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"],.dropdown-submenu:hover>a>[class^="icon-"],.dropdown-submenu:focus>a>[class^="icon-"],.dropdown-submenu:hover>a>[class*=" icon-"],.dropdown-submenu:focus>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{width:16px;background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu>li>a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus,.dropdown-submenu:hover>a,.dropdown-submenu:focus>a{color:#fff;text-decoration:none;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{color:#fff;text-decoration:none;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;outline:0;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu>.disabled>a,.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{color:#999}.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{text-decoration:none;cursor:default;background-color:transparent;background-image:none;filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:""}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover>.dropdown-menu{display:block}.dropup .dropdown-submenu>.dropdown-menu{top:auto;bottom:0;margin-top:0;margin-bottom:-2px;-webkit-border-radius:5px 5px 5px 0;-moz-border-radius:5px 5px 5px 0;border-radius:5px 5px 5px 0}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown-submenu.pull-left{float:none}.dropdown-submenu.pull-left>.dropdown-menu{left:-100%;margin-left:10px;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{z-index:1051;margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover,.close:focus{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 12px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #ccc;*border:0;border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-bottom-color:#b3b3b3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:focus,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover,.btn:focus{color:#333;text-decoration:none;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:11px 19px;font-size:17.5px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.btn-large [class^="icon-"],.btn-large [class*=" icon-"]{margin-top:4px}.btn-small{padding:2px 10px;font-size:11.9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.btn-small [class^="icon-"],.btn-small [class*=" icon-"]{margin-top:0}.btn-mini [class^="icon-"],.btn-mini [class*=" icon-"]{margin-top:-1px}.btn-mini{padding:0 6px;font-size:10.5px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}input[type="submit"].btn-block,input[type="reset"].btn-block,input[type="button"].btn-block{width:100%}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-moz-linear-gradient(top,#08c,#04c);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:focus,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:focus,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:focus,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-moz-linear-gradient(top,#62c462,#51a351);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:focus,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:focus,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-moz-linear-gradient(top,#444,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:focus,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active,.btn-link[disabled]{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover,.btn-link:focus{color:#005580;text-decoration:underline;background-color:transparent}.btn-link[disabled]:hover,.btn-link[disabled]:focus{color:#333;text-decoration:none}.btn-group{position:relative;display:inline-block;*display:inline;*margin-left:.3em;font-size:0;white-space:nowrap;vertical-align:middle;*zoom:1}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar>.btn+.btn,.btn-toolbar>.btn-group+.btn,.btn-toolbar>.btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu,.btn-group>.popover{font-size:14px}.btn-group>.btn-mini{font-size:10.5px}.btn-group>.btn-small{font-size:11.9px}.btn-group>.btn-large{font-size:17.5px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.btn-mini .caret,.btn-small .caret{margin-top:8px}.dropup .btn-large .caret{border-bottom-width:5px}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical>.btn{display:block;float:none;max-width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical>.btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical>.btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical>.btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical>.btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical>.btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert,.alert h4{color:#c09853}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-success h4{color:#468847}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-danger h4,.alert-error h4{color:#b94a48}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-info h4{color:#3a87ad}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover,.nav>li>a:focus{text-decoration:none;background-color:#eee}.nav>li>a>img{max-width:none}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover,.nav-list>.active>a:focus{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"],.nav-list [class*=" icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover,.nav-tabs>li>a:focus{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover,.nav-tabs>.active>a:focus{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover,.nav-pills>.active>a:focus{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover,.nav-tabs.nav-stacked>li>a:focus{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret,.nav .dropdown-toggle:focus .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover,.nav>.dropdown.active>a:focus{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover,.nav>li.dropdown.open.active>a:focus{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret,.nav li.dropdown.open a:focus .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover,.tabs-stacked .open>a:focus{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover,.tabs-below>.nav-tabs>li>a:focus{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover,.tabs-below>.nav-tabs>.active>a:focus{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover,.tabs-left>.nav-tabs>li>a:focus{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover,.tabs-left>.nav-tabs .active>a:focus{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover,.tabs-right>.nav-tabs>li>a:focus{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover,.tabs-right>.nav-tabs .active>a:focus{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover,.nav>.disabled>a:focus{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#f2f2f2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#f2f2f2));background-image:-webkit-linear-gradient(top,#fff,#f2f2f2);background-image:-o-linear-gradient(top,#fff,#f2f2f2);background-image:linear-gradient(to bottom,#fff,#f2f2f2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fff2f2f2',GradientType=0);*zoom:1;-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar-inner:before,.navbar-inner:after{display:table;line-height:0;content:""}.navbar-inner:after{clear:both}.navbar .container{width:auto}.nav-collapse.collapse{height:auto;overflow:visible}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#777;text-shadow:0 1px 0 #fff}.navbar .brand:hover,.navbar .brand:focus{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px;color:#777}.navbar-link{color:#777}.navbar-link:hover,.navbar-link:focus{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:5px}.navbar .btn-group .btn,.navbar .input-prepend .btn,.navbar .input-append .btn,.navbar .input-prepend .btn-group,.navbar .input-append .btn-group{margin-top:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:5px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{border-width:0 0 1px}.navbar-fixed-bottom .navbar-inner{border-width:1px 0 0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 10px rgba(0,0,0,0.1);box-shadow:0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:0 -1px 10px rgba(0,0,0,0.1);box-shadow:0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right;margin-right:0}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#777;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:focus,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown>a:hover .caret,.navbar .nav li.dropdown>a:focus .caret{border-top-color:#333;border-bottom-color:#333}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#777;border-bottom-color:#777}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover,.navbar-inverse .brand:focus,.navbar-inverse .nav>li>a:focus{color:#fff}.navbar-inverse .brand{color:#999}.navbar-inverse .navbar-text{color:#999}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover,.navbar-inverse .navbar-link:focus{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>a:hover .caret,.navbar-inverse .nav li.dropdown>a:focus .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-moz-linear-gradient(top,#151515,#040404);background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:focus,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb>li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb>li>.divider{padding:0 5px;color:#ccc}.breadcrumb>.active{color:#999}.pagination{margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination ul>li{display:inline}.pagination ul>li>a,.pagination ul>li>span{float:left;padding:4px 12px;line-height:20px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination ul>li>a:hover,.pagination ul>li>a:focus,.pagination ul>.active>a,.pagination ul>.active>span{background-color:#f5f5f5}.pagination ul>.active>a,.pagination ul>.active>span{color:#999;cursor:default}.pagination ul>.disabled>span,.pagination ul>.disabled>a,.pagination ul>.disabled>a:hover,.pagination ul>.disabled>a:focus{color:#999;cursor:default;background-color:transparent}.pagination ul>li:first-child>a,.pagination ul>li:first-child>span{border-left-width:1px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.pagination ul>li:last-child>a,.pagination ul>li:last-child>span{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pagination-large ul>li>a,.pagination-large ul>li>span{padding:11px 19px;font-size:17.5px}.pagination-large ul>li:first-child>a,.pagination-large ul>li:first-child>span{-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.pagination-large ul>li:last-child>a,.pagination-large ul>li:last-child>span{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.pagination-mini ul>li:first-child>a,.pagination-small ul>li:first-child>a,.pagination-mini ul>li:first-child>span,.pagination-small ul>li:first-child>span{-webkit-border-bottom-left-radius:3px;border-bottom-left-radius:3px;-webkit-border-top-left-radius:3px;border-top-left-radius:3px;-moz-border-radius-bottomleft:3px;-moz-border-radius-topleft:3px}.pagination-mini ul>li:last-child>a,.pagination-small ul>li:last-child>a,.pagination-mini ul>li:last-child>span,.pagination-small ul>li:last-child>span{-webkit-border-top-right-radius:3px;border-top-right-radius:3px;-webkit-border-bottom-right-radius:3px;border-bottom-right-radius:3px;-moz-border-radius-topright:3px;-moz-border-radius-bottomright:3px}.pagination-small ul>li>a,.pagination-small ul>li>span{padding:2px 10px;font-size:11.9px}.pagination-mini ul>li>a,.pagination-mini ul>li>span{padding:0 6px;font-size:10.5px}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager li>a,.pager li>span{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager li>a:hover,.pager li>a:focus{text-decoration:none;background-color:#f5f5f5}.pager .next>a,.pager .next>span{float:right}.pager .previous>a,.pager .previous>span{float:left}.pager .disabled>a,.pager .disabled>a:hover,.pager .disabled>a:focus,.pager .disabled>span{color:#999;cursor:default;background-color:#fff}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:10%;left:50%;z-index:1050;width:560px;margin-left:-280px;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;outline:0;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:10%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{position:relative;max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.modal-footer .btn-block+.btn-block{margin-left:0}.tooltip{position:absolute;z-index:1030;display:block;font-size:11px;line-height:1.4;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{padding:5px 0;margin-top:-3px}.tooltip.right{padding:0 5px;margin-left:3px}.tooltip.bottom{padding:5px 0;margin-top:3px}.tooltip.left{padding:0 5px;margin-left:-3px}.tooltip-inner{max-width:200px;padding:8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;max-width:276px;padding:1px;text-align:left;white-space:normal;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-top:-10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-left:-10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-title:empty{display:none}.popover-content{padding:9px 14px}.popover .arrow,.popover .arrow:after{position:absolute;display:block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow{border-width:11px}.popover .arrow:after{border-width:10px;content:""}.popover.top .arrow{bottom:-11px;left:50%;margin-left:-11px;border-top-color:#999;border-top-color:rgba(0,0,0,0.25);border-bottom-width:0}.popover.top .arrow:after{bottom:1px;margin-left:-10px;border-top-color:#fff;border-bottom-width:0}.popover.right .arrow{top:50%;left:-11px;margin-top:-11px;border-right-color:#999;border-right-color:rgba(0,0,0,0.25);border-left-width:0}.popover.right .arrow:after{bottom:-10px;left:1px;border-right-color:#fff;border-left-width:0}.popover.bottom .arrow{top:-11px;left:50%;margin-left:-11px;border-bottom-color:#999;border-bottom-color:rgba(0,0,0,0.25);border-top-width:0}.popover.bottom .arrow:after{top:1px;margin-left:-10px;border-bottom-color:#fff;border-top-width:0}.popover.left .arrow{top:50%;right:-11px;margin-top:-11px;border-left-color:#999;border-left-color:rgba(0,0,0,0.25);border-right-width:0}.popover.left .arrow:after{right:1px;bottom:-10px;border-left-color:#fff;border-right-width:0}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover,a.thumbnail:focus{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.media,.media-body{overflow:hidden;*overflow:visible;zoom:1}.media,.media .media{margin-top:15px}.media:first-child{margin-top:0}.media-object{display:block}.media-heading{margin:0 0 5px}.media>.pull-left{margin-right:10px}.media>.pull-right{margin-left:10px}.media-list{margin-left:0;list-style:none}.label,.badge{display:inline-block;padding:2px 4px;font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding-right:9px;padding-left:9px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}.label:empty,.badge:empty{display:none}a.label:hover,a.label:focus,a.badge:hover,a.badge:focus{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel-inner>.item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel-inner>.item>img,.carousel-inner>.item>a>img{display:block;line-height:1}.carousel-inner>.active,.carousel-inner>.next,.carousel-inner>.prev{display:block}.carousel-inner>.active{left:0}.carousel-inner>.next,.carousel-inner>.prev{position:absolute;top:0;width:100%}.carousel-inner>.next{left:100%}.carousel-inner>.prev{left:-100%}.carousel-inner>.next.left,.carousel-inner>.prev.right{left:0}.carousel-inner>.active.left{left:-100%}.carousel-inner>.active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover,.carousel-control:focus{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-indicators{position:absolute;top:15px;right:15px;z-index:5;margin:0;list-style:none}.carousel-indicators li{display:block;float:left;width:10px;height:10px;margin-left:5px;text-indent:-999px;background-color:#ccc;background-color:rgba(255,255,255,0.25);border-radius:5px}.carousel-indicators .active{background-color:#fff}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;font-size:18px;font-weight:200;line-height:30px;color:inherit;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit li{line-height:30px}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed}
+ */
+.clearfix{*zoom:1;}.clearfix:before,.clearfix:after{display:table;content:"";line-height:0;}
+.clearfix:after{clear:both;}
+.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0;}
+.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;}
+article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block;}
+audio,canvas,video{display:inline-block;*display:inline;*zoom:1;}
+audio:not([controls]){display:none;}
+html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%;}
+a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px;}
+a:hover,a:active{outline:0;}
+sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline;}
+sup{top:-0.5em;}
+sub{bottom:-0.25em;}
+img{max-width:100%;width:auto\9;height:auto;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic;}
+#map_canvas img,.google-maps img{max-width:none;}
+button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle;}
+button,input{*overflow:visible;line-height:normal;}
+button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0;}
+button,html input[type="button"],input[type="reset"],input[type="submit"]{-webkit-appearance:button;cursor:pointer;}
+label,select,button,input[type="button"],input[type="reset"],input[type="submit"],input[type="radio"],input[type="checkbox"]{cursor:pointer;}
+input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield;}
+input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none;}
+textarea{overflow:auto;vertical-align:top;}
+@media print{*{text-shadow:none !important;color:#000 !important;background:transparent !important;box-shadow:none !important;} a,a:visited{text-decoration:underline;} a[href]:after{content:" (" attr(href) ")";} abbr[title]:after{content:" (" attr(title) ")";} .ir a:after,a[href^="javascript:"]:after,a[href^="#"]:after{content:"";} pre,blockquote{border:1px solid #999;page-break-inside:avoid;} thead{display:table-header-group;} tr,img{page-break-inside:avoid;} img{max-width:100% !important;} @page {margin:0.5cm;}p,h2,h3{orphans:3;widows:3;} h2,h3{page-break-after:avoid;}}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333333;background-color:#ffffff;}
+a{color:#0088cc;text-decoration:none;}
+a:hover,a:focus{color:#005580;text-decoration:underline;}
+.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;}
+.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0, 0, 0, 0.2);-webkit-box-shadow:0 1px 3px rgba(0, 0, 0, 0.1);-moz-box-shadow:0 1px 3px rgba(0, 0, 0, 0.1);box-shadow:0 1px 3px rgba(0, 0, 0, 0.1);}
+.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px;}
+.row{margin-left:-20px;*zoom:1;}.row:before,.row:after{display:table;content:"";line-height:0;}
+.row:after{clear:both;}
+[class*="span"]{float:left;min-height:1px;margin-left:20px;}
+.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px;}
+.span12{width:940px;}
+.span11{width:860px;}
+.span10{width:780px;}
+.span9{width:700px;}
+.span8{width:620px;}
+.span7{width:540px;}
+.span6{width:460px;}
+.span5{width:380px;}
+.span4{width:300px;}
+.span3{width:220px;}
+.span2{width:140px;}
+.span1{width:60px;}
+.offset12{margin-left:980px;}
+.offset11{margin-left:900px;}
+.offset10{margin-left:820px;}
+.offset9{margin-left:740px;}
+.offset8{margin-left:660px;}
+.offset7{margin-left:580px;}
+.offset6{margin-left:500px;}
+.offset5{margin-left:420px;}
+.offset4{margin-left:340px;}
+.offset3{margin-left:260px;}
+.offset2{margin-left:180px;}
+.offset1{margin-left:100px;}
+.row-fluid{width:100%;*zoom:1;}.row-fluid:before,.row-fluid:after{display:table;content:"";line-height:0;}
+.row-fluid:after{clear:both;}
+.row-fluid [class*="span"]{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;float:left;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;}
+.row-fluid [class*="span"]:first-child{margin-left:0;}
+.row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.127659574468085%;}
+.row-fluid .span12{width:100%;*width:99.94680851063829%;}
+.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%;}
+.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%;}
+.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%;}
+.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%;}
+.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%;}
+.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%;}
+.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%;}
+.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%;}
+.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%;}
+.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%;}
+.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%;}
+.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%;}
+.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%;}
+.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%;}
+.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%;}
+.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%;}
+.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%;}
+.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%;}
+.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%;}
+.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%;}
+.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%;}
+.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%;}
+.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%;}
+.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%;}
+.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%;}
+.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%;}
+.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%;}
+.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%;}
+.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%;}
+.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%;}
+.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%;}
+.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%;}
+.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%;}
+.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%;}
+.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%;}
+[class*="span"].hide,.row-fluid [class*="span"].hide{display:none;}
+[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right;}
+.container{margin-right:auto;margin-left:auto;*zoom:1;}.container:before,.container:after{display:table;content:"";line-height:0;}
+.container:after{clear:both;}
+.container-fluid{padding-right:20px;padding-left:20px;*zoom:1;}.container-fluid:before,.container-fluid:after{display:table;content:"";line-height:0;}
+.container-fluid:after{clear:both;}
+p{margin:0 0 10px;}
+.lead{margin-bottom:20px;font-size:21px;font-weight:200;line-height:30px;}
+small{font-size:85%;}
+strong{font-weight:bold;}
+em{font-style:italic;}
+cite{font-style:normal;}
+.muted{color:#999999;}
+a.muted:hover,a.muted:focus{color:#808080;}
+.text-warning{color:#c09853;}
+a.text-warning:hover,a.text-warning:focus{color:#a47e3c;}
+.text-error{color:#b94a48;}
+a.text-error:hover,a.text-error:focus{color:#953b39;}
+.text-info{color:#3a87ad;}
+a.text-info:hover,a.text-info:focus{color:#2d6987;}
+.text-success{color:#468847;}
+a.text-success:hover,a.text-success:focus{color:#356635;}
+.text-left{text-align:left;}
+.text-right{text-align:right;}
+.text-center{text-align:center;}
+h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:20px;color:inherit;text-rendering:optimizelegibility;}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999999;}
+h1,h2,h3{line-height:40px;}
+h1{font-size:38.5px;}
+h2{font-size:31.5px;}
+h3{font-size:24.5px;}
+h4{font-size:17.5px;}
+h5{font-size:14px;}
+h6{font-size:11.9px;}
+h1 small{font-size:24.5px;}
+h2 small{font-size:17.5px;}
+h3 small{font-size:14px;}
+h4 small{font-size:14px;}
+.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eeeeee;}
+ul,ol{padding:0;margin:0 0 10px 25px;}
+ul ul,ul ol,ol ol,ol ul{margin-bottom:0;}
+li{line-height:20px;}
+ul.unstyled,ol.unstyled{margin-left:0;list-style:none;}
+ul.inline,ol.inline{margin-left:0;list-style:none;}ul.inline>li,ol.inline>li{display:inline-block;*display:inline;*zoom:1;padding-left:5px;padding-right:5px;}
+dl{margin-bottom:20px;}
+dt,dd{line-height:20px;}
+dt{font-weight:bold;}
+dd{margin-left:10px;}
+.dl-horizontal{*zoom:1;}.dl-horizontal:before,.dl-horizontal:after{display:table;content:"";line-height:0;}
+.dl-horizontal:after{clear:both;}
+.dl-horizontal dt{float:left;width:160px;clear:left;text-align:right;overflow:hidden;text-overflow:ellipsis;white-space:nowrap;}
+.dl-horizontal dd{margin-left:180px;}
+hr{margin:20px 0;border:0;border-top:1px solid #eeeeee;border-bottom:1px solid #ffffff;}
+abbr[title],abbr[data-original-title]{cursor:help;border-bottom:1px dotted #999999;}
+abbr.initialism{font-size:90%;text-transform:uppercase;}
+blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eeeeee;}blockquote p{margin-bottom:0;font-size:17.5px;font-weight:300;line-height:1.25;}
+blockquote small{display:block;line-height:20px;color:#999999;}blockquote small:before{content:'\2014 \00A0';}
+blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eeeeee;border-left:0;}blockquote.pull-right p,blockquote.pull-right small{text-align:right;}
+blockquote.pull-right small:before{content:'';}
+blockquote.pull-right small:after{content:'\00A0 \2014';}
+q:before,q:after,blockquote:before,blockquote:after{content:"";}
+address{display:block;margin-bottom:20px;font-style:normal;line-height:20px;}
+code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;}
+code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8;white-space:nowrap;}
+pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0, 0, 0, 0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}pre.prettyprint{margin-bottom:20px;}
+pre code{padding:0;color:inherit;white-space:pre;white-space:pre-wrap;background-color:transparent;border:0;}
+.pre-scrollable{max-height:340px;overflow-y:scroll;}
+.label,.badge{display:inline-block;padding:2px 4px;font-size:11.844px;font-weight:bold;line-height:14px;color:#ffffff;vertical-align:baseline;white-space:nowrap;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#999999;}
+.label{-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;}
+.badge{padding-left:9px;padding-right:9px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px;}
+.label:empty,.badge:empty{display:none;}
+a.label:hover,a.label:focus,a.badge:hover,a.badge:focus{color:#ffffff;text-decoration:none;cursor:pointer;}
+.label-important,.badge-important{background-color:#b94a48;}
+.label-important[href],.badge-important[href]{background-color:#953b39;}
+.label-warning,.badge-warning{background-color:#f89406;}
+.label-warning[href],.badge-warning[href]{background-color:#c67605;}
+.label-success,.badge-success{background-color:#468847;}
+.label-success[href],.badge-success[href]{background-color:#356635;}
+.label-info,.badge-info{background-color:#3a87ad;}
+.label-info[href],.badge-info[href]{background-color:#2d6987;}
+.label-inverse,.badge-inverse{background-color:#333333;}
+.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a;}
+.btn .label,.btn .badge{position:relative;top:-1px;}
+.btn-mini .label,.btn-mini .badge{top:0;}
+table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0;}
+.table{width:100%;margin-bottom:20px;}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #dddddd;}
+.table th{font-weight:bold;}
+.table thead th{vertical-align:bottom;}
+.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0;}
+.table tbody+tbody{border-top:2px solid #dddddd;}
+.table .table{background-color:#ffffff;}
+.table-condensed th,.table-condensed td{padding:4px 5px;}
+.table-bordered{border:1px solid #dddddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}.table-bordered th,.table-bordered td{border-left:1px solid #dddddd;}
+.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0;}
+.table-bordered thead:first-child tr:first-child>th:first-child,.table-bordered tbody:first-child tr:first-child>td:first-child,.table-bordered tbody:first-child tr:first-child>th:first-child{-webkit-border-top-left-radius:4px;-moz-border-radius-topleft:4px;border-top-left-radius:4px;}
+.table-bordered thead:first-child tr:first-child>th:last-child,.table-bordered tbody:first-child tr:first-child>td:last-child,.table-bordered tbody:first-child tr:first-child>th:last-child{-webkit-border-top-right-radius:4px;-moz-border-radius-topright:4px;border-top-right-radius:4px;}
+.table-bordered thead:last-child tr:last-child>th:first-child,.table-bordered tbody:last-child tr:last-child>td:first-child,.table-bordered tbody:last-child tr:last-child>th:first-child,.table-bordered tfoot:last-child tr:last-child>td:first-child,.table-bordered tfoot:last-child tr:last-child>th:first-child{-webkit-border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px;border-bottom-left-radius:4px;}
+.table-bordered thead:last-child tr:last-child>th:last-child,.table-bordered tbody:last-child tr:last-child>td:last-child,.table-bordered tbody:last-child tr:last-child>th:last-child,.table-bordered tfoot:last-child tr:last-child>td:last-child,.table-bordered tfoot:last-child tr:last-child>th:last-child{-webkit-border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px;border-bottom-right-radius:4px;}
+.table-bordered tfoot+tbody:last-child tr:last-child td:first-child{-webkit-border-bottom-left-radius:0;-moz-border-radius-bottomleft:0;border-bottom-left-radius:0;}
+.table-bordered tfoot+tbody:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:0;-moz-border-radius-bottomright:0;border-bottom-right-radius:0;}
+.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;-moz-border-radius-topleft:4px;border-top-left-radius:4px;}
+.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;-moz-border-radius-topright:4px;border-top-right-radius:4px;}
+.table-striped tbody>tr:nth-child(odd)>td,.table-striped tbody>tr:nth-child(odd)>th{background-color:#f9f9f9;}
+.table-hover tbody tr:hover>td,.table-hover tbody tr:hover>th{background-color:#f5f5f5;}
+table td[class*="span"],table th[class*="span"],.row-fluid table td[class*="span"],.row-fluid table th[class*="span"]{display:table-cell;float:none;margin-left:0;}
+.table td.span1,.table th.span1{float:none;width:44px;margin-left:0;}
+.table td.span2,.table th.span2{float:none;width:124px;margin-left:0;}
+.table td.span3,.table th.span3{float:none;width:204px;margin-left:0;}
+.table td.span4,.table th.span4{float:none;width:284px;margin-left:0;}
+.table td.span5,.table th.span5{float:none;width:364px;margin-left:0;}
+.table td.span6,.table th.span6{float:none;width:444px;margin-left:0;}
+.table td.span7,.table th.span7{float:none;width:524px;margin-left:0;}
+.table td.span8,.table th.span8{float:none;width:604px;margin-left:0;}
+.table td.span9,.table th.span9{float:none;width:684px;margin-left:0;}
+.table td.span10,.table th.span10{float:none;width:764px;margin-left:0;}
+.table td.span11,.table th.span11{float:none;width:844px;margin-left:0;}
+.table td.span12,.table th.span12{float:none;width:924px;margin-left:0;}
+.table tbody tr.success>td{background-color:#dff0d8;}
+.table tbody tr.error>td{background-color:#f2dede;}
+.table tbody tr.warning>td{background-color:#fcf8e3;}
+.table tbody tr.info>td{background-color:#d9edf7;}
+.table-hover tbody tr.success:hover>td{background-color:#d0e9c6;}
+.table-hover tbody tr.error:hover>td{background-color:#ebcccc;}
+.table-hover tbody tr.warning:hover>td{background-color:#faf2cc;}
+.table-hover tbody tr.info:hover>td{background-color:#c4e3f3;}
+form{margin:0 0 20px;}
+fieldset{padding:0;margin:0;border:0;}
+legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333333;border:0;border-bottom:1px solid #e5e5e5;}legend small{font-size:15px;color:#999999;}
+label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px;}
+input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;}
+label{display:block;margin-bottom:5px;}
+select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:10px;font-size:14px;line-height:20px;color:#555555;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;vertical-align:middle;}
+input,textarea,.uneditable-input{width:206px;}
+textarea{height:auto;}
+textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#ffffff;border:1px solid #cccccc;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-webkit-transition:border linear .2s, box-shadow linear .2s;-moz-transition:border linear .2s, box-shadow linear .2s;-o-transition:border linear .2s, box-shadow linear .2s;transition:border linear .2s, box-shadow linear .2s;}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82, 168, 236, 0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(82,168,236,.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(82,168,236,.6);box-shadow:inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(82,168,236,.6);}
+input[type="radio"],input[type="checkbox"]{margin:4px 0 0;*margin-top:0;margin-top:1px \9;line-height:normal;}
+input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto;}
+select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px;}
+select{width:220px;border:1px solid #cccccc;background-color:#ffffff;}
+select[multiple],select[size]{height:auto;}
+select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px;}
+.uneditable-input,.uneditable-textarea{color:#999999;background-color:#fcfcfc;border-color:#cccccc;-webkit-box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.025);-moz-box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.025);box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.025);cursor:not-allowed;}
+.uneditable-input{overflow:hidden;white-space:nowrap;}
+.uneditable-textarea{width:auto;height:auto;}
+input:-moz-placeholder,textarea:-moz-placeholder{color:#999999;}
+input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999999;}
+input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999999;}
+.radio,.checkbox{min-height:20px;padding-left:20px;}
+.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-20px;}
+.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px;}
+.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle;}
+.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px;}
+.input-mini{width:60px;}
+.input-small{width:90px;}
+.input-medium{width:150px;}
+.input-large{width:210px;}
+.input-xlarge{width:270px;}
+.input-xxlarge{width:530px;}
+input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0;}
+.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block;}
+input,textarea,.uneditable-input{margin-left:0;}
+.controls-row [class*="span"]+[class*="span"]{margin-left:20px;}
+input.span12,textarea.span12,.uneditable-input.span12{width:926px;}
+input.span11,textarea.span11,.uneditable-input.span11{width:846px;}
+input.span10,textarea.span10,.uneditable-input.span10{width:766px;}
+input.span9,textarea.span9,.uneditable-input.span9{width:686px;}
+input.span8,textarea.span8,.uneditable-input.span8{width:606px;}
+input.span7,textarea.span7,.uneditable-input.span7{width:526px;}
+input.span6,textarea.span6,.uneditable-input.span6{width:446px;}
+input.span5,textarea.span5,.uneditable-input.span5{width:366px;}
+input.span4,textarea.span4,.uneditable-input.span4{width:286px;}
+input.span3,textarea.span3,.uneditable-input.span3{width:206px;}
+input.span2,textarea.span2,.uneditable-input.span2{width:126px;}
+input.span1,textarea.span1,.uneditable-input.span1{width:46px;}
+.controls-row{*zoom:1;}.controls-row:before,.controls-row:after{display:table;content:"";line-height:0;}
+.controls-row:after{clear:both;}
+.controls-row [class*="span"],.row-fluid .controls-row [class*="span"]{float:left;}
+.controls-row .checkbox[class*="span"],.controls-row .radio[class*="span"]{padding-top:5px;}
+input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eeeeee;}
+input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent;}
+.control-group.warning .control-label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853;}
+.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;}
+.control-group.warning input,.control-group.warning select,.control-group.warning textarea{border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);}.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #dbc59e;}
+.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853;}
+.control-group.error .control-label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48;}
+.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;}
+.control-group.error input,.control-group.error select,.control-group.error textarea{border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);}.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #d59392;}
+.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48;}
+.control-group.success .control-label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847;}
+.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;}
+.control-group.success input,.control-group.success select,.control-group.success textarea{border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);}.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7aba7b;}
+.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847;}
+.control-group.info .control-label,.control-group.info .help-block,.control-group.info .help-inline{color:#3a87ad;}
+.control-group.info .checkbox,.control-group.info .radio,.control-group.info input,.control-group.info select,.control-group.info textarea{color:#3a87ad;}
+.control-group.info input,.control-group.info select,.control-group.info textarea{border-color:#3a87ad;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);}.control-group.info input:focus,.control-group.info select:focus,.control-group.info textarea:focus{border-color:#2d6987;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7ab5d3;-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7ab5d3;box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7ab5d3;}
+.control-group.info .input-prepend .add-on,.control-group.info .input-append .add-on{color:#3a87ad;background-color:#d9edf7;border-color:#3a87ad;}
+input:focus:invalid,textarea:focus:invalid,select:focus:invalid{color:#b94a48;border-color:#ee5f5b;}input:focus:invalid:focus,textarea:focus:invalid:focus,select:focus:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7;}
+.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1;}.form-actions:before,.form-actions:after{display:table;content:"";line-height:0;}
+.form-actions:after{clear:both;}
+.help-block,.help-inline{color:#595959;}
+.help-block{display:block;margin-bottom:10px;}
+.help-inline{display:inline-block;*display:inline;*zoom:1;vertical-align:middle;padding-left:5px;}
+.input-append,.input-prepend{display:inline-block;margin-bottom:10px;vertical-align:middle;font-size:0;white-space:nowrap;}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input,.input-append .dropdown-menu,.input-prepend .dropdown-menu,.input-append .popover,.input-prepend .popover{font-size:14px;}
+.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;vertical-align:top;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2;}
+.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #ffffff;background-color:#eeeeee;border:1px solid #ccc;}
+.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn,.input-append .btn-group>.dropdown-toggle,.input-prepend .btn-group>.dropdown-toggle{vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;}
+.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546;}
+.input-prepend .add-on,.input-prepend .btn{margin-right:-1px;}
+.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px;}
+.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px;}.input-append input+.btn-group .btn:last-child,.input-append select+.btn-group .btn:last-child,.input-append .uneditable-input+.btn-group .btn:last-child{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;}
+.input-append .add-on,.input-append .btn,.input-append .btn-group{margin-left:-1px;}
+.input-append .add-on:last-child,.input-append .btn:last-child,.input-append .btn-group:last-child>.dropdown-toggle{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;}
+.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;}.input-prepend.input-append input+.btn-group .btn,.input-prepend.input-append select+.btn-group .btn,.input-prepend.input-append .uneditable-input+.btn-group .btn{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;}
+.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px;}
+.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;}
+.input-prepend.input-append .btn-group:first-child{margin-left:0;}
+input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px;}
+.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;}
+.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px;}
+.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0;}
+.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0;}
+.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px;}
+.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;*zoom:1;margin-bottom:0;vertical-align:middle;}
+.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none;}
+.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block;}
+.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0;}
+.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle;}
+.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0;}
+.control-group{margin-bottom:10px;}
+legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate;}
+.form-horizontal .control-group{margin-bottom:20px;*zoom:1;}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;content:"";line-height:0;}
+.form-horizontal .control-group:after{clear:both;}
+.form-horizontal .control-label{float:left;width:160px;padding-top:5px;text-align:right;}
+.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:180px;*margin-left:0;}.form-horizontal .controls:first-child{*padding-left:180px;}
+.form-horizontal .help-block{margin-bottom:0;}
+.form-horizontal input+.help-block,.form-horizontal select+.help-block,.form-horizontal textarea+.help-block,.form-horizontal .uneditable-input+.help-block,.form-horizontal .input-prepend+.help-block,.form-horizontal .input-append+.help-block{margin-top:10px;}
+.form-horizontal .form-actions{padding-left:180px;}
+.btn{display:inline-block;*display:inline;*zoom:1;padding:4px 12px;margin-bottom:0;font-size:14px;line-height:20px;text-align:center;vertical-align:middle;cursor:pointer;color:#333333;text-shadow:0 1px 1px rgba(255, 255, 255, 0.75);background-color:#f5f5f5;background-image:-moz-linear-gradient(top, #ffffff, #e6e6e6);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), to(#e6e6e6));background-image:-webkit-linear-gradient(top, #ffffff, #e6e6e6);background-image:-o-linear-gradient(top, #ffffff, #e6e6e6);background-image:linear-gradient(to bottom, #ffffff, #e6e6e6);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#ffe6e6e6', GradientType=0);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#e6e6e6;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);border:1px solid #cccccc;*border:0;border-bottom-color:#b3b3b3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;*margin-left:.3em;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);box-shadow:inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);}.btn:hover,.btn:focus,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333333;background-color:#e6e6e6;*background-color:#d9d9d9;}
+.btn:active,.btn.active{background-color:#cccccc \9;}
+.btn:first-child{*margin-left:0;}
+.btn:hover,.btn:focus{color:#333333;text-decoration:none;background-position:0 -15px;-webkit-transition:background-position 0.1s linear;-moz-transition:background-position 0.1s linear;-o-transition:background-position 0.1s linear;transition:background-position 0.1s linear;}
+.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px;}
+.btn.active,.btn:active{background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);}
+.btn.disabled,.btn[disabled]{cursor:default;background-image:none;opacity:0.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none;}
+.btn-large{padding:11px 19px;font-size:17.5px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;}
+.btn-large [class^="icon-"],.btn-large [class*=" icon-"]{margin-top:4px;}
+.btn-small{padding:2px 10px;font-size:11.9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;}
+.btn-small [class^="icon-"],.btn-small [class*=" icon-"]{margin-top:0;}
+.btn-mini [class^="icon-"],.btn-mini [class*=" icon-"]{margin-top:-1px;}
+.btn-mini{padding:0 6px;font-size:10.5px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;}
+.btn-block{display:block;width:100%;padding-left:0;padding-right:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;}
+.btn-block+.btn-block{margin-top:5px;}
+input[type="submit"].btn-block,input[type="reset"].btn-block,input[type="button"].btn-block{width:100%;}
+.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255, 255, 255, 0.75);}
+.btn-primary{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#006dcc;background-image:-moz-linear-gradient(top, #0088cc, #0044cc);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0044cc));background-image:-webkit-linear-gradient(top, #0088cc, #0044cc);background-image:-o-linear-gradient(top, #0088cc, #0044cc);background-image:linear-gradient(to bottom, #0088cc, #0044cc);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0044cc', GradientType=0);border-color:#0044cc #0044cc #002a80;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#0044cc;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-primary:hover,.btn-primary:focus,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#ffffff;background-color:#0044cc;*background-color:#003bb3;}
+.btn-primary:active,.btn-primary.active{background-color:#003399 \9;}
+.btn-warning{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#faa732;background-image:-moz-linear-gradient(top, #fbb450, #f89406);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#fbb450), to(#f89406));background-image:-webkit-linear-gradient(top, #fbb450, #f89406);background-image:-o-linear-gradient(top, #fbb450, #f89406);background-image:linear-gradient(to bottom, #fbb450, #f89406);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450', endColorstr='#fff89406', GradientType=0);border-color:#f89406 #f89406 #ad6704;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#f89406;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-warning:hover,.btn-warning:focus,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#ffffff;background-color:#f89406;*background-color:#df8505;}
+.btn-warning:active,.btn-warning.active{background-color:#c67605 \9;}
+.btn-danger{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#da4f49;background-image:-moz-linear-gradient(top, #ee5f5b, #bd362f);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#ee5f5b), to(#bd362f));background-image:-webkit-linear-gradient(top, #ee5f5b, #bd362f);background-image:-o-linear-gradient(top, #ee5f5b, #bd362f);background-image:linear-gradient(to bottom, #ee5f5b, #bd362f);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b', endColorstr='#ffbd362f', GradientType=0);border-color:#bd362f #bd362f #802420;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#bd362f;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-danger:hover,.btn-danger:focus,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#ffffff;background-color:#bd362f;*background-color:#a9302a;}
+.btn-danger:active,.btn-danger.active{background-color:#942a25 \9;}
+.btn-success{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#5bb75b;background-image:-moz-linear-gradient(top, #62c462, #51a351);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#62c462), to(#51a351));background-image:-webkit-linear-gradient(top, #62c462, #51a351);background-image:-o-linear-gradient(top, #62c462, #51a351);background-image:linear-gradient(to bottom, #62c462, #51a351);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462', endColorstr='#ff51a351', GradientType=0);border-color:#51a351 #51a351 #387038;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#51a351;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-success:hover,.btn-success:focus,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#ffffff;background-color:#51a351;*background-color:#499249;}
+.btn-success:active,.btn-success.active{background-color:#408140 \9;}
+.btn-info{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#49afcd;background-image:-moz-linear-gradient(top, #5bc0de, #2f96b4);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#5bc0de), to(#2f96b4));background-image:-webkit-linear-gradient(top, #5bc0de, #2f96b4);background-image:-o-linear-gradient(top, #5bc0de, #2f96b4);background-image:linear-gradient(to bottom, #5bc0de, #2f96b4);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff2f96b4', GradientType=0);border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#2f96b4;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-info:hover,.btn-info:focus,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#ffffff;background-color:#2f96b4;*background-color:#2a85a0;}
+.btn-info:active,.btn-info.active{background-color:#24748c \9;}
+.btn-inverse{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#363636;background-image:-moz-linear-gradient(top, #444444, #222222);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#444444), to(#222222));background-image:-webkit-linear-gradient(top, #444444, #222222);background-image:-o-linear-gradient(top, #444444, #222222);background-image:linear-gradient(to bottom, #444444, #222222);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff444444', endColorstr='#ff222222', GradientType=0);border-color:#222222 #222222 #000000;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#222222;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-inverse:hover,.btn-inverse:focus,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#ffffff;background-color:#222222;*background-color:#151515;}
+.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9;}
+button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px;}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0;}
+button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px;}
+button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px;}
+button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px;}
+.btn-link,.btn-link:active,.btn-link[disabled]{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none;}
+.btn-link{border-color:transparent;cursor:pointer;color:#0088cc;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;}
+.btn-link:hover,.btn-link:focus{color:#005580;text-decoration:underline;background-color:transparent;}
+.btn-link[disabled]:hover,.btn-link[disabled]:focus{color:#333333;text-decoration:none;}
+[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat;margin-top:1px;}
+.icon-white,.nav-pills>.active>a>[class^="icon-"],.nav-pills>.active>a>[class*=" icon-"],.nav-list>.active>a>[class^="icon-"],.nav-list>.active>a>[class*=" icon-"],.navbar-inverse .nav>.active>a>[class^="icon-"],.navbar-inverse .nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:focus>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>li>a:focus>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"],.dropdown-submenu:hover>a>[class^="icon-"],.dropdown-submenu:focus>a>[class^="icon-"],.dropdown-submenu:hover>a>[class*=" icon-"],.dropdown-submenu:focus>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png");}
+.icon-glass{background-position:0 0;}
+.icon-music{background-position:-24px 0;}
+.icon-search{background-position:-48px 0;}
+.icon-envelope{background-position:-72px 0;}
+.icon-heart{background-position:-96px 0;}
+.icon-star{background-position:-120px 0;}
+.icon-star-empty{background-position:-144px 0;}
+.icon-user{background-position:-168px 0;}
+.icon-film{background-position:-192px 0;}
+.icon-th-large{background-position:-216px 0;}
+.icon-th{background-position:-240px 0;}
+.icon-th-list{background-position:-264px 0;}
+.icon-ok{background-position:-288px 0;}
+.icon-remove{background-position:-312px 0;}
+.icon-zoom-in{background-position:-336px 0;}
+.icon-zoom-out{background-position:-360px 0;}
+.icon-off{background-position:-384px 0;}
+.icon-signal{background-position:-408px 0;}
+.icon-cog{background-position:-432px 0;}
+.icon-trash{background-position:-456px 0;}
+.icon-home{background-position:0 -24px;}
+.icon-file{background-position:-24px -24px;}
+.icon-time{background-position:-48px -24px;}
+.icon-road{background-position:-72px -24px;}
+.icon-download-alt{background-position:-96px -24px;}
+.icon-download{background-position:-120px -24px;}
+.icon-upload{background-position:-144px -24px;}
+.icon-inbox{background-position:-168px -24px;}
+.icon-play-circle{background-position:-192px -24px;}
+.icon-repeat{background-position:-216px -24px;}
+.icon-refresh{background-position:-240px -24px;}
+.icon-list-alt{background-position:-264px -24px;}
+.icon-lock{background-position:-287px -24px;}
+.icon-flag{background-position:-312px -24px;}
+.icon-headphones{background-position:-336px -24px;}
+.icon-volume-off{background-position:-360px -24px;}
+.icon-volume-down{background-position:-384px -24px;}
+.icon-volume-up{background-position:-408px -24px;}
+.icon-qrcode{background-position:-432px -24px;}
+.icon-barcode{background-position:-456px -24px;}
+.icon-tag{background-position:0 -48px;}
+.icon-tags{background-position:-25px -48px;}
+.icon-book{background-position:-48px -48px;}
+.icon-bookmark{background-position:-72px -48px;}
+.icon-print{background-position:-96px -48px;}
+.icon-camera{background-position:-120px -48px;}
+.icon-font{background-position:-144px -48px;}
+.icon-bold{background-position:-167px -48px;}
+.icon-italic{background-position:-192px -48px;}
+.icon-text-height{background-position:-216px -48px;}
+.icon-text-width{background-position:-240px -48px;}
+.icon-align-left{background-position:-264px -48px;}
+.icon-align-center{background-position:-288px -48px;}
+.icon-align-right{background-position:-312px -48px;}
+.icon-align-justify{background-position:-336px -48px;}
+.icon-list{background-position:-360px -48px;}
+.icon-indent-left{background-position:-384px -48px;}
+.icon-indent-right{background-position:-408px -48px;}
+.icon-facetime-video{background-position:-432px -48px;}
+.icon-picture{background-position:-456px -48px;}
+.icon-pencil{background-position:0 -72px;}
+.icon-map-marker{background-position:-24px -72px;}
+.icon-adjust{background-position:-48px -72px;}
+.icon-tint{background-position:-72px -72px;}
+.icon-edit{background-position:-96px -72px;}
+.icon-share{background-position:-120px -72px;}
+.icon-check{background-position:-144px -72px;}
+.icon-move{background-position:-168px -72px;}
+.icon-step-backward{background-position:-192px -72px;}
+.icon-fast-backward{background-position:-216px -72px;}
+.icon-backward{background-position:-240px -72px;}
+.icon-play{background-position:-264px -72px;}
+.icon-pause{background-position:-288px -72px;}
+.icon-stop{background-position:-312px -72px;}
+.icon-forward{background-position:-336px -72px;}
+.icon-fast-forward{background-position:-360px -72px;}
+.icon-step-forward{background-position:-384px -72px;}
+.icon-eject{background-position:-408px -72px;}
+.icon-chevron-left{background-position:-432px -72px;}
+.icon-chevron-right{background-position:-456px -72px;}
+.icon-plus-sign{background-position:0 -96px;}
+.icon-minus-sign{background-position:-24px -96px;}
+.icon-remove-sign{background-position:-48px -96px;}
+.icon-ok-sign{background-position:-72px -96px;}
+.icon-question-sign{background-position:-96px -96px;}
+.icon-info-sign{background-position:-120px -96px;}
+.icon-screenshot{background-position:-144px -96px;}
+.icon-remove-circle{background-position:-168px -96px;}
+.icon-ok-circle{background-position:-192px -96px;}
+.icon-ban-circle{background-position:-216px -96px;}
+.icon-arrow-left{background-position:-240px -96px;}
+.icon-arrow-right{background-position:-264px -96px;}
+.icon-arrow-up{background-position:-289px -96px;}
+.icon-arrow-down{background-position:-312px -96px;}
+.icon-share-alt{background-position:-336px -96px;}
+.icon-resize-full{background-position:-360px -96px;}
+.icon-resize-small{background-position:-384px -96px;}
+.icon-plus{background-position:-408px -96px;}
+.icon-minus{background-position:-433px -96px;}
+.icon-asterisk{background-position:-456px -96px;}
+.icon-exclamation-sign{background-position:0 -120px;}
+.icon-gift{background-position:-24px -120px;}
+.icon-leaf{background-position:-48px -120px;}
+.icon-fire{background-position:-72px -120px;}
+.icon-eye-open{background-position:-96px -120px;}
+.icon-eye-close{background-position:-120px -120px;}
+.icon-warning-sign{background-position:-144px -120px;}
+.icon-plane{background-position:-168px -120px;}
+.icon-calendar{background-position:-192px -120px;}
+.icon-random{background-position:-216px -120px;width:16px;}
+.icon-comment{background-position:-240px -120px;}
+.icon-magnet{background-position:-264px -120px;}
+.icon-chevron-up{background-position:-288px -120px;}
+.icon-chevron-down{background-position:-313px -119px;}
+.icon-retweet{background-position:-336px -120px;}
+.icon-shopping-cart{background-position:-360px -120px;}
+.icon-folder-close{background-position:-384px -120px;width:16px;}
+.icon-folder-open{background-position:-408px -120px;width:16px;}
+.icon-resize-vertical{background-position:-432px -119px;}
+.icon-resize-horizontal{background-position:-456px -118px;}
+.icon-hdd{background-position:0 -144px;}
+.icon-bullhorn{background-position:-24px -144px;}
+.icon-bell{background-position:-48px -144px;}
+.icon-certificate{background-position:-72px -144px;}
+.icon-thumbs-up{background-position:-96px -144px;}
+.icon-thumbs-down{background-position:-120px -144px;}
+.icon-hand-right{background-position:-144px -144px;}
+.icon-hand-left{background-position:-168px -144px;}
+.icon-hand-up{background-position:-192px -144px;}
+.icon-hand-down{background-position:-216px -144px;}
+.icon-circle-arrow-right{background-position:-240px -144px;}
+.icon-circle-arrow-left{background-position:-264px -144px;}
+.icon-circle-arrow-up{background-position:-288px -144px;}
+.icon-circle-arrow-down{background-position:-312px -144px;}
+.icon-globe{background-position:-336px -144px;}
+.icon-wrench{background-position:-360px -144px;}
+.icon-tasks{background-position:-384px -144px;}
+.icon-filter{background-position:-408px -144px;}
+.icon-briefcase{background-position:-432px -144px;}
+.icon-fullscreen{background-position:-456px -144px;}
+.btn-group{position:relative;display:inline-block;*display:inline;*zoom:1;font-size:0;vertical-align:middle;white-space:nowrap;*margin-left:.3em;}.btn-group:first-child{*margin-left:0;}
+.btn-group+.btn-group{margin-left:5px;}
+.btn-toolbar{font-size:0;margin-top:10px;margin-bottom:10px;}.btn-toolbar>.btn+.btn,.btn-toolbar>.btn-group+.btn,.btn-toolbar>.btn+.btn-group{margin-left:5px;}
+.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;}
+.btn-group>.btn+.btn{margin-left:-1px;}
+.btn-group>.btn,.btn-group>.dropdown-menu,.btn-group>.popover{font-size:14px;}
+.btn-group>.btn-mini{font-size:10.5px;}
+.btn-group>.btn-small{font-size:11.9px;}
+.btn-group>.btn-large{font-size:17.5px;}
+.btn-group>.btn:first-child{margin-left:0;-webkit-border-top-left-radius:4px;-moz-border-radius-topleft:4px;border-top-left-radius:4px;-webkit-border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px;border-bottom-left-radius:4px;}
+.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;-moz-border-radius-topright:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px;border-bottom-right-radius:4px;}
+.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-top-left-radius:6px;-moz-border-radius-topleft:6px;border-top-left-radius:6px;-webkit-border-bottom-left-radius:6px;-moz-border-radius-bottomleft:6px;border-bottom-left-radius:6px;}
+.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;-moz-border-radius-topright:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;-moz-border-radius-bottomright:6px;border-bottom-right-radius:6px;}
+.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2;}
+.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0;}
+.btn-group>.btn+.dropdown-toggle{padding-left:8px;padding-right:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,.125), inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,.125), inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);box-shadow:inset 1px 0 0 rgba(255,255,255,.125), inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);*padding-top:5px;*padding-bottom:5px;}
+.btn-group>.btn-mini+.dropdown-toggle{padding-left:5px;padding-right:5px;*padding-top:2px;*padding-bottom:2px;}
+.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px;}
+.btn-group>.btn-large+.dropdown-toggle{padding-left:12px;padding-right:12px;*padding-top:7px;*padding-bottom:7px;}
+.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);}
+.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6;}
+.btn-group.open .btn-primary.dropdown-toggle{background-color:#0044cc;}
+.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406;}
+.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f;}
+.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351;}
+.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4;}
+.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222222;}
+.btn .caret{margin-top:8px;margin-left:0;}
+.btn-large .caret{margin-top:6px;}
+.btn-large .caret{border-left-width:5px;border-right-width:5px;border-top-width:5px;}
+.btn-mini .caret,.btn-small .caret{margin-top:8px;}
+.dropup .btn-large .caret{border-bottom-width:5px;}
+.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#ffffff;border-bottom-color:#ffffff;}
+.btn-group-vertical{display:inline-block;*display:inline;*zoom:1;}
+.btn-group-vertical>.btn{display:block;float:none;max-width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;}
+.btn-group-vertical>.btn+.btn{margin-left:0;margin-top:-1px;}
+.btn-group-vertical>.btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0;}
+.btn-group-vertical>.btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px;}
+.btn-group-vertical>.btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0;}
+.btn-group-vertical>.btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;}
+.nav{margin-left:0;margin-bottom:20px;list-style:none;}
+.nav>li>a{display:block;}
+.nav>li>a:hover,.nav>li>a:focus{text-decoration:none;background-color:#eeeeee;}
+.nav>li>a>img{max-width:none;}
+.nav>.pull-right{float:right;}
+.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999999;text-shadow:0 1px 0 rgba(255, 255, 255, 0.5);text-transform:uppercase;}
+.nav li+.nav-header{margin-top:9px;}
+.nav-list{padding-left:15px;padding-right:15px;margin-bottom:0;}
+.nav-list>li>a,.nav-list .nav-header{margin-left:-15px;margin-right:-15px;text-shadow:0 1px 0 rgba(255, 255, 255, 0.5);}
+.nav-list>li>a{padding:3px 15px;}
+.nav-list>.active>a,.nav-list>.active>a:hover,.nav-list>.active>a:focus{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.2);background-color:#0088cc;}
+.nav-list [class^="icon-"],.nav-list [class*=" icon-"]{margin-right:2px;}
+.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #ffffff;}
+.nav-tabs,.nav-pills{*zoom:1;}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;content:"";line-height:0;}
+.nav-tabs:after,.nav-pills:after{clear:both;}
+.nav-tabs>li,.nav-pills>li{float:left;}
+.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px;}
+.nav-tabs{border-bottom:1px solid #ddd;}
+.nav-tabs>li{margin-bottom:-1px;}
+.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0;}.nav-tabs>li>a:hover,.nav-tabs>li>a:focus{border-color:#eeeeee #eeeeee #dddddd;}
+.nav-tabs>.active>a,.nav-tabs>.active>a:hover,.nav-tabs>.active>a:focus{color:#555555;background-color:#ffffff;border:1px solid #ddd;border-bottom-color:transparent;cursor:default;}
+.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px;}
+.nav-pills>.active>a,.nav-pills>.active>a:hover,.nav-pills>.active>a:focus{color:#ffffff;background-color:#0088cc;}
+.nav-stacked>li{float:none;}
+.nav-stacked>li>a{margin-right:0;}
+.nav-tabs.nav-stacked{border-bottom:0;}
+.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;}
+.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;-moz-border-radius-topright:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;-moz-border-radius-topleft:4px;border-top-left-radius:4px;}
+.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px;border-bottom-left-radius:4px;}
+.nav-tabs.nav-stacked>li>a:hover,.nav-tabs.nav-stacked>li>a:focus{border-color:#ddd;z-index:2;}
+.nav-pills.nav-stacked>li>a{margin-bottom:3px;}
+.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px;}
+.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;}
+.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;}
+.nav .dropdown-toggle .caret{border-top-color:#0088cc;border-bottom-color:#0088cc;margin-top:6px;}
+.nav .dropdown-toggle:hover .caret,.nav .dropdown-toggle:focus .caret{border-top-color:#005580;border-bottom-color:#005580;}
+.nav-tabs .dropdown-toggle .caret{margin-top:8px;}
+.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff;}
+.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555555;border-bottom-color:#555555;}
+.nav>.dropdown.active>a:hover,.nav>.dropdown.active>a:focus{cursor:pointer;}
+.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover,.nav>li.dropdown.open.active>a:focus{color:#ffffff;background-color:#999999;border-color:#999999;}
+.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret,.nav li.dropdown.open a:focus .caret{border-top-color:#ffffff;border-bottom-color:#ffffff;opacity:1;filter:alpha(opacity=100);}
+.tabs-stacked .open>a:hover,.tabs-stacked .open>a:focus{border-color:#999999;}
+.tabbable{*zoom:1;}.tabbable:before,.tabbable:after{display:table;content:"";line-height:0;}
+.tabbable:after{clear:both;}
+.tab-content{overflow:auto;}
+.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0;}
+.tab-content>.tab-pane,.pill-content>.pill-pane{display:none;}
+.tab-content>.active,.pill-content>.active{display:block;}
+.tabs-below>.nav-tabs{border-top:1px solid #ddd;}
+.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0;}
+.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px;}.tabs-below>.nav-tabs>li>a:hover,.tabs-below>.nav-tabs>li>a:focus{border-bottom-color:transparent;border-top-color:#ddd;}
+.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover,.tabs-below>.nav-tabs>.active>a:focus{border-color:transparent #ddd #ddd #ddd;}
+.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none;}
+.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px;}
+.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd;}
+.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px;}
+.tabs-left>.nav-tabs>li>a:hover,.tabs-left>.nav-tabs>li>a:focus{border-color:#eeeeee #dddddd #eeeeee #eeeeee;}
+.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover,.tabs-left>.nav-tabs .active>a:focus{border-color:#ddd transparent #ddd #ddd;*border-right-color:#ffffff;}
+.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd;}
+.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;}
+.tabs-right>.nav-tabs>li>a:hover,.tabs-right>.nav-tabs>li>a:focus{border-color:#eeeeee #eeeeee #eeeeee #dddddd;}
+.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover,.tabs-right>.nav-tabs .active>a:focus{border-color:#ddd #ddd #ddd transparent;*border-left-color:#ffffff;}
+.nav>.disabled>a{color:#999999;}
+.nav>.disabled>a:hover,.nav>.disabled>a:focus{text-decoration:none;background-color:transparent;cursor:default;}
+.navbar{overflow:visible;margin-bottom:20px;*position:relative;*z-index:2;}
+.navbar-inner{min-height:40px;padding-left:20px;padding-right:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top, #ffffff, #f2f2f2);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), to(#f2f2f2));background-image:-webkit-linear-gradient(top, #ffffff, #f2f2f2);background-image:-o-linear-gradient(top, #ffffff, #f2f2f2);background-image:linear-gradient(to bottom, #ffffff, #f2f2f2);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#fff2f2f2', GradientType=0);border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 4px rgba(0, 0, 0, 0.065);-moz-box-shadow:0 1px 4px rgba(0, 0, 0, 0.065);box-shadow:0 1px 4px rgba(0, 0, 0, 0.065);*zoom:1;}.navbar-inner:before,.navbar-inner:after{display:table;content:"";line-height:0;}
+.navbar-inner:after{clear:both;}
+.navbar .container{width:auto;}
+.nav-collapse.collapse{height:auto;overflow:visible;}
+.navbar .brand{float:left;display:block;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#777777;text-shadow:0 1px 0 #ffffff;}.navbar .brand:hover,.navbar .brand:focus{text-decoration:none;}
+.navbar-text{margin-bottom:0;line-height:40px;color:#777777;}
+.navbar-link{color:#777777;}.navbar-link:hover,.navbar-link:focus{color:#333333;}
+.navbar .divider-vertical{height:40px;margin:0 9px;border-left:1px solid #f2f2f2;border-right:1px solid #ffffff;}
+.navbar .btn,.navbar .btn-group{margin-top:5px;}
+.navbar .btn-group .btn,.navbar .input-prepend .btn,.navbar .input-append .btn,.navbar .input-prepend .btn-group,.navbar .input-append .btn-group{margin-top:0;}
+.navbar-form{margin-bottom:0;*zoom:1;}.navbar-form:before,.navbar-form:after{display:table;content:"";line-height:0;}
+.navbar-form:after{clear:both;}
+.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px;}
+.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0;}
+.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px;}
+.navbar-form .input-append,.navbar-form .input-prepend{margin-top:5px;white-space:nowrap;}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0;}
+.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0;}.navbar-search .search-query{margin-bottom:0;padding:4px 14px;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px;}
+.navbar-static-top{position:static;margin-bottom:0;}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;}
+.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0;}
+.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{border-width:0 0 1px;}
+.navbar-fixed-bottom .navbar-inner{border-width:1px 0 0;}
+.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-left:0;padding-right:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;}
+.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px;}
+.navbar-fixed-top{top:0;}
+.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:0 1px 10px rgba(0,0,0,.1);-moz-box-shadow:0 1px 10px rgba(0,0,0,.1);box-shadow:0 1px 10px rgba(0,0,0,.1);}
+.navbar-fixed-bottom{bottom:0;}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:0 -1px 10px rgba(0,0,0,.1);-moz-box-shadow:0 -1px 10px rgba(0,0,0,.1);box-shadow:0 -1px 10px rgba(0,0,0,.1);}
+.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0;}
+.navbar .nav.pull-right{float:right;margin-right:0;}
+.navbar .nav>li{float:left;}
+.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#777777;text-decoration:none;text-shadow:0 1px 0 #ffffff;}
+.navbar .nav .dropdown-toggle .caret{margin-top:8px;}
+.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{background-color:transparent;color:#333333;text-decoration:none;}
+.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0, 0, 0, 0.125);-moz-box-shadow:inset 0 3px 8px rgba(0, 0, 0, 0.125);box-shadow:inset 0 3px 8px rgba(0, 0, 0, 0.125);}
+.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-left:5px;margin-right:5px;color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#ededed;background-image:-moz-linear-gradient(top, #f2f2f2, #e5e5e5);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#f2f2f2), to(#e5e5e5));background-image:-webkit-linear-gradient(top, #f2f2f2, #e5e5e5);background-image:-o-linear-gradient(top, #f2f2f2, #e5e5e5);background-image:linear-gradient(to bottom, #f2f2f2, #e5e5e5);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2f2f2', endColorstr='#ffe5e5e5', GradientType=0);border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#e5e5e5;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.075);box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.075);}.navbar .btn-navbar:hover,.navbar .btn-navbar:focus,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#ffffff;background-color:#e5e5e5;*background-color:#d9d9d9;}
+.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#cccccc \9;}
+.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0, 0, 0, 0.25);-moz-box-shadow:0 1px 0 rgba(0, 0, 0, 0.25);box-shadow:0 1px 0 rgba(0, 0, 0, 0.25);}
+.btn-navbar .icon-bar+.icon-bar{margin-top:3px;}
+.navbar .nav>li>.dropdown-menu:before{content:'';display:inline-block;border-left:7px solid transparent;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-bottom-color:rgba(0, 0, 0, 0.2);position:absolute;top:-7px;left:9px;}
+.navbar .nav>li>.dropdown-menu:after{content:'';display:inline-block;border-left:6px solid transparent;border-right:6px solid transparent;border-bottom:6px solid #ffffff;position:absolute;top:-6px;left:10px;}
+.navbar-fixed-bottom .nav>li>.dropdown-menu:before{border-top:7px solid #ccc;border-top-color:rgba(0, 0, 0, 0.2);border-bottom:0;bottom:-7px;top:auto;}
+.navbar-fixed-bottom .nav>li>.dropdown-menu:after{border-top:6px solid #ffffff;border-bottom:0;bottom:-6px;top:auto;}
+.navbar .nav li.dropdown>a:hover .caret,.navbar .nav li.dropdown>a:focus .caret{border-top-color:#333333;border-bottom-color:#333333;}
+.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{background-color:#e5e5e5;color:#555555;}
+.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#777777;border-bottom-color:#777777;}
+.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555555;border-bottom-color:#555555;}
+.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{left:auto;right:0;}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{left:auto;right:12px;}
+.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{left:auto;right:13px;}
+.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{left:auto;right:100%;margin-left:0;margin-right:-1px;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px;}
+.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top, #222222, #111111);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#222222), to(#111111));background-image:-webkit-linear-gradient(top, #222222, #111111);background-image:-o-linear-gradient(top, #222222, #111111);background-image:linear-gradient(to bottom, #222222, #111111);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff222222', endColorstr='#ff111111', GradientType=0);border-color:#252525;}
+.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999999;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover,.navbar-inverse .brand:focus,.navbar-inverse .nav>li>a:focus{color:#ffffff;}
+.navbar-inverse .brand{color:#999999;}
+.navbar-inverse .navbar-text{color:#999999;}
+.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{background-color:transparent;color:#ffffff;}
+.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#ffffff;background-color:#111111;}
+.navbar-inverse .navbar-link{color:#999999;}.navbar-inverse .navbar-link:hover,.navbar-inverse .navbar-link:focus{color:#ffffff;}
+.navbar-inverse .divider-vertical{border-left-color:#111111;border-right-color:#222222;}
+.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{background-color:#111111;color:#ffffff;}
+.navbar-inverse .nav li.dropdown>a:hover .caret,.navbar-inverse .nav li.dropdown>a:focus .caret{border-top-color:#ffffff;border-bottom-color:#ffffff;}
+.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999999;border-bottom-color:#999999;}
+.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#ffffff;border-bottom-color:#ffffff;}
+.navbar-inverse .navbar-search .search-query{color:#ffffff;background-color:#515151;border-color:#111111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,.1), 0 1px 0 rgba(255,255,255,.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,.1), 0 1px 0 rgba(255,255,255,.15);box-shadow:inset 0 1px 2px rgba(0,0,0,.1), 0 1px 0 rgba(255,255,255,.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none;}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#cccccc;}
+.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#cccccc;}
+.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#cccccc;}
+.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333333;text-shadow:0 1px 0 #ffffff;background-color:#ffffff;border:0;-webkit-box-shadow:0 0 3px rgba(0, 0, 0, 0.15);-moz-box-shadow:0 0 3px rgba(0, 0, 0, 0.15);box-shadow:0 0 3px rgba(0, 0, 0, 0.15);outline:0;}
+.navbar-inverse .btn-navbar{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#0e0e0e;background-image:-moz-linear-gradient(top, #151515, #040404);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#151515), to(#040404));background-image:-webkit-linear-gradient(top, #151515, #040404);background-image:-o-linear-gradient(top, #151515, #040404);background-image:linear-gradient(to bottom, #151515, #040404);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff151515', endColorstr='#ff040404', GradientType=0);border-color:#040404 #040404 #000000;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#040404;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:focus,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#ffffff;background-color:#040404;*background-color:#000000;}
+.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000000 \9;}
+.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}.breadcrumb>li{display:inline-block;*display:inline;*zoom:1;text-shadow:0 1px 0 #ffffff;}.breadcrumb>li>.divider{padding:0 5px;color:#ccc;}
+.breadcrumb>.active{color:#999999;}
+.pagination{margin:20px 0;}
+.pagination ul{display:inline-block;*display:inline;*zoom:1;margin-left:0;margin-bottom:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 2px rgba(0, 0, 0, 0.05);-moz-box-shadow:0 1px 2px rgba(0, 0, 0, 0.05);box-shadow:0 1px 2px rgba(0, 0, 0, 0.05);}
+.pagination ul>li{display:inline;}
+.pagination ul>li>a,.pagination ul>li>span{float:left;padding:4px 12px;line-height:20px;text-decoration:none;background-color:#ffffff;border:1px solid #dddddd;border-left-width:0;}
+.pagination ul>li>a:hover,.pagination ul>li>a:focus,.pagination ul>.active>a,.pagination ul>.active>span{background-color:#f5f5f5;}
+.pagination ul>.active>a,.pagination ul>.active>span{color:#999999;cursor:default;}
+.pagination ul>.disabled>span,.pagination ul>.disabled>a,.pagination ul>.disabled>a:hover,.pagination ul>.disabled>a:focus{color:#999999;background-color:transparent;cursor:default;}
+.pagination ul>li:first-child>a,.pagination ul>li:first-child>span{border-left-width:1px;-webkit-border-top-left-radius:4px;-moz-border-radius-topleft:4px;border-top-left-radius:4px;-webkit-border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px;border-bottom-left-radius:4px;}
+.pagination ul>li:last-child>a,.pagination ul>li:last-child>span{-webkit-border-top-right-radius:4px;-moz-border-radius-topright:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px;border-bottom-right-radius:4px;}
+.pagination-centered{text-align:center;}
+.pagination-right{text-align:right;}
+.pagination-large ul>li>a,.pagination-large ul>li>span{padding:11px 19px;font-size:17.5px;}
+.pagination-large ul>li:first-child>a,.pagination-large ul>li:first-child>span{-webkit-border-top-left-radius:6px;-moz-border-radius-topleft:6px;border-top-left-radius:6px;-webkit-border-bottom-left-radius:6px;-moz-border-radius-bottomleft:6px;border-bottom-left-radius:6px;}
+.pagination-large ul>li:last-child>a,.pagination-large ul>li:last-child>span{-webkit-border-top-right-radius:6px;-moz-border-radius-topright:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;-moz-border-radius-bottomright:6px;border-bottom-right-radius:6px;}
+.pagination-mini ul>li:first-child>a,.pagination-small ul>li:first-child>a,.pagination-mini ul>li:first-child>span,.pagination-small ul>li:first-child>span{-webkit-border-top-left-radius:3px;-moz-border-radius-topleft:3px;border-top-left-radius:3px;-webkit-border-bottom-left-radius:3px;-moz-border-radius-bottomleft:3px;border-bottom-left-radius:3px;}
+.pagination-mini ul>li:last-child>a,.pagination-small ul>li:last-child>a,.pagination-mini ul>li:last-child>span,.pagination-small ul>li:last-child>span{-webkit-border-top-right-radius:3px;-moz-border-radius-topright:3px;border-top-right-radius:3px;-webkit-border-bottom-right-radius:3px;-moz-border-radius-bottomright:3px;border-bottom-right-radius:3px;}
+.pagination-small ul>li>a,.pagination-small ul>li>span{padding:2px 10px;font-size:11.9px;}
+.pagination-mini ul>li>a,.pagination-mini ul>li>span{padding:0 6px;font-size:10.5px;}
+.pager{margin:20px 0;list-style:none;text-align:center;*zoom:1;}.pager:before,.pager:after{display:table;content:"";line-height:0;}
+.pager:after{clear:both;}
+.pager li{display:inline;}
+.pager li>a,.pager li>span{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px;}
+.pager li>a:hover,.pager li>a:focus{text-decoration:none;background-color:#f5f5f5;}
+.pager .next>a,.pager .next>span{float:right;}
+.pager .previous>a,.pager .previous>span{float:left;}
+.pager .disabled>a,.pager .disabled>a:hover,.pager .disabled>a:focus,.pager .disabled>span{color:#999999;background-color:#fff;cursor:default;}
+.thumbnails{margin-left:-20px;list-style:none;*zoom:1;}.thumbnails:before,.thumbnails:after{display:table;content:"";line-height:0;}
+.thumbnails:after{clear:both;}
+.row-fluid .thumbnails{margin-left:0;}
+.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px;}
+.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0, 0, 0, 0.055);-moz-box-shadow:0 1px 3px rgba(0, 0, 0, 0.055);box-shadow:0 1px 3px rgba(0, 0, 0, 0.055);-webkit-transition:all 0.2s ease-in-out;-moz-transition:all 0.2s ease-in-out;-o-transition:all 0.2s ease-in-out;transition:all 0.2s ease-in-out;}
+a.thumbnail:hover,a.thumbnail:focus{border-color:#0088cc;-webkit-box-shadow:0 1px 4px rgba(0, 105, 214, 0.25);-moz-box-shadow:0 1px 4px rgba(0, 105, 214, 0.25);box-shadow:0 1px 4px rgba(0, 105, 214, 0.25);}
+.thumbnail>img{display:block;max-width:100%;margin-left:auto;margin-right:auto;}
+.thumbnail .caption{padding:9px;color:#555555;}
+.alert{padding:8px 35px 8px 14px;margin-bottom:20px;text-shadow:0 1px 0 rgba(255, 255, 255, 0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}
+.alert,.alert h4{color:#c09853;}
+.alert h4{margin:0;}
+.alert .close{position:relative;top:-2px;right:-21px;line-height:20px;}
+.alert-success{background-color:#dff0d8;border-color:#d6e9c6;color:#468847;}
+.alert-success h4{color:#468847;}
+.alert-danger,.alert-error{background-color:#f2dede;border-color:#eed3d7;color:#b94a48;}
+.alert-danger h4,.alert-error h4{color:#b94a48;}
+.alert-info{background-color:#d9edf7;border-color:#bce8f1;color:#3a87ad;}
+.alert-info h4{color:#3a87ad;}
+.alert-block{padding-top:14px;padding-bottom:14px;}
+.alert-block>p,.alert-block>ul{margin-bottom:0;}
+.alert-block p+p{margin-top:5px;}
+@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0;} to{background-position:0 0;}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0;} to{background-position:0 0;}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0;} to{background-position:0 0;}}@-o-keyframes progress-bar-stripes{from{background-position:0 0;} to{background-position:40px 0;}}@keyframes progress-bar-stripes{from{background-position:40px 0;} to{background-position:0 0;}}.progress{overflow:hidden;height:20px;margin-bottom:20px;background-color:#f7f7f7;background-image:-moz-linear-gradient(top, #f5f5f5, #f9f9f9);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#f5f5f5), to(#f9f9f9));background-image:-webkit-linear-gradient(top, #f5f5f5, #f9f9f9);background-image:-o-linear-gradient(top, #f5f5f5, #f9f9f9);background-image:linear-gradient(to bottom, #f5f5f5, #f9f9f9);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#fff9f9f9', GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.1);-moz-box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.1);box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.1);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}
+.progress .bar{width:0%;height:100%;color:#ffffff;float:left;font-size:12px;text-align:center;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top, #149bdf, #0480be);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#149bdf), to(#0480be));background-image:-webkit-linear-gradient(top, #149bdf, #0480be);background-image:-o-linear-gradient(top, #149bdf, #0480be);background-image:linear-gradient(to bottom, #149bdf, #0480be);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff149bdf', endColorstr='#ff0480be', GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0, 0, 0, 0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0, 0, 0, 0.15);box-shadow:inset 0 -1px 0 rgba(0, 0, 0, 0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width 0.6s ease;-moz-transition:width 0.6s ease;-o-transition:width 0.6s ease;transition:width 0.6s ease;}
+.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,.15), inset 0 -1px 0 rgba(0,0,0,.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,.15), inset 0 -1px 0 rgba(0,0,0,.15);box-shadow:inset 1px 0 0 rgba(0,0,0,.15), inset 0 -1px 0 rgba(0,0,0,.15);}
+.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent));background-image:-webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px;}
+.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite;}
+.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top, #ee5f5b, #c43c35);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#ee5f5b), to(#c43c35));background-image:-webkit-linear-gradient(top, #ee5f5b, #c43c35);background-image:-o-linear-gradient(top, #ee5f5b, #c43c35);background-image:linear-gradient(to bottom, #ee5f5b, #c43c35);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b', endColorstr='#ffc43c35', GradientType=0);}
+.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent));background-image:-webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);}
+.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top, #62c462, #57a957);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#62c462), to(#57a957));background-image:-webkit-linear-gradient(top, #62c462, #57a957);background-image:-o-linear-gradient(top, #62c462, #57a957);background-image:linear-gradient(to bottom, #62c462, #57a957);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462', endColorstr='#ff57a957', GradientType=0);}
+.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent));background-image:-webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);}
+.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top, #5bc0de, #339bb9);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#5bc0de), to(#339bb9));background-image:-webkit-linear-gradient(top, #5bc0de, #339bb9);background-image:-o-linear-gradient(top, #5bc0de, #339bb9);background-image:linear-gradient(to bottom, #5bc0de, #339bb9);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff339bb9', GradientType=0);}
+.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent));background-image:-webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);}
+.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top, #fbb450, #f89406);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#fbb450), to(#f89406));background-image:-webkit-linear-gradient(top, #fbb450, #f89406);background-image:-o-linear-gradient(top, #fbb450, #f89406);background-image:linear-gradient(to bottom, #fbb450, #f89406);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450', endColorstr='#fff89406', GradientType=0);}
+.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent));background-image:-webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);}
+.hero-unit{padding:60px;margin-bottom:30px;font-size:18px;font-weight:200;line-height:30px;color:inherit;background-color:#eeeeee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;color:inherit;letter-spacing:-1px;}
+.hero-unit li{line-height:30px;}
+.media,.media-body{overflow:hidden;*overflow:visible;zoom:1;}
+.media,.media .media{margin-top:15px;}
+.media:first-child{margin-top:0;}
+.media-object{display:block;}
+.media-heading{margin:0 0 5px;}
+.media>.pull-left{margin-right:10px;}
+.media>.pull-right{margin-left:10px;}
+.media-list{margin-left:0;list-style:none;}
+.tooltip{position:absolute;z-index:1030;display:block;visibility:visible;font-size:11px;line-height:1.4;opacity:0;filter:alpha(opacity=0);}.tooltip.in{opacity:0.8;filter:alpha(opacity=80);}
+.tooltip.top{margin-top:-3px;padding:5px 0;}
+.tooltip.right{margin-left:3px;padding:0 5px;}
+.tooltip.bottom{margin-top:3px;padding:5px 0;}
+.tooltip.left{margin-left:-3px;padding:0 5px;}
+.tooltip-inner{max-width:200px;padding:8px;color:#ffffff;text-align:center;text-decoration:none;background-color:#000000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}
+.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid;}
+.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-width:5px 5px 0;border-top-color:#000000;}
+.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-width:5px 5px 5px 0;border-right-color:#000000;}
+.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-width:5px 0 5px 5px;border-left-color:#000000;}
+.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-width:0 5px 5px;border-bottom-color:#000000;}
+.popover{position:absolute;top:0;left:0;z-index:1010;display:none;max-width:276px;padding:1px;text-align:left;background-color:#ffffff;-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box;border:1px solid #ccc;border:1px solid rgba(0, 0, 0, 0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);-moz-box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);white-space:normal;}.popover.top{margin-top:-10px;}
+.popover.right{margin-left:10px;}
+.popover.bottom{margin-top:10px;}
+.popover.left{margin-left:-10px;}
+.popover-title{margin:0;padding:8px 14px;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0;}.popover-title:empty{display:none;}
+.popover-content{padding:9px 14px;}
+.popover .arrow,.popover .arrow:after{position:absolute;display:block;width:0;height:0;border-color:transparent;border-style:solid;}
+.popover .arrow{border-width:11px;}
+.popover .arrow:after{border-width:10px;content:"";}
+.popover.top .arrow{left:50%;margin-left:-11px;border-bottom-width:0;border-top-color:#999;border-top-color:rgba(0, 0, 0, 0.25);bottom:-11px;}.popover.top .arrow:after{bottom:1px;margin-left:-10px;border-bottom-width:0;border-top-color:#ffffff;}
+.popover.right .arrow{top:50%;left:-11px;margin-top:-11px;border-left-width:0;border-right-color:#999;border-right-color:rgba(0, 0, 0, 0.25);}.popover.right .arrow:after{left:1px;bottom:-10px;border-left-width:0;border-right-color:#ffffff;}
+.popover.bottom .arrow{left:50%;margin-left:-11px;border-top-width:0;border-bottom-color:#999;border-bottom-color:rgba(0, 0, 0, 0.25);top:-11px;}.popover.bottom .arrow:after{top:1px;margin-left:-10px;border-top-width:0;border-bottom-color:#ffffff;}
+.popover.left .arrow{top:50%;right:-11px;margin-top:-11px;border-right-width:0;border-left-color:#999;border-left-color:rgba(0, 0, 0, 0.25);}.popover.left .arrow:after{right:1px;border-right-width:0;border-left-color:#ffffff;bottom:-10px;}
+.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000000;}.modal-backdrop.fade{opacity:0;}
+.modal-backdrop,.modal-backdrop.fade.in{opacity:0.8;filter:alpha(opacity=80);}
+.modal{position:fixed;top:10%;left:50%;z-index:1050;width:560px;margin-left:-280px;background-color:#ffffff;border:1px solid #999;border:1px solid rgba(0, 0, 0, 0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0, 0, 0, 0.3);-moz-box-shadow:0 3px 7px rgba(0, 0, 0, 0.3);box-shadow:0 3px 7px rgba(0, 0, 0, 0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box;outline:none;}.modal.fade{-webkit-transition:opacity .3s linear, top .3s ease-out;-moz-transition:opacity .3s linear, top .3s ease-out;-o-transition:opacity .3s linear, top .3s ease-out;transition:opacity .3s linear, top .3s ease-out;top:-25%;}
+.modal.fade.in{top:10%;}
+.modal-header{padding:9px 15px;border-bottom:1px solid #eee;}.modal-header .close{margin-top:2px;}
+.modal-header h3{margin:0;line-height:30px;}
+.modal-body{position:relative;overflow-y:auto;max-height:400px;padding:15px;}
+.modal-form{margin-bottom:0;}
+.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;-webkit-box-shadow:inset 0 1px 0 #ffffff;-moz-box-shadow:inset 0 1px 0 #ffffff;box-shadow:inset 0 1px 0 #ffffff;*zoom:1;}.modal-footer:before,.modal-footer:after{display:table;content:"";line-height:0;}
+.modal-footer:after{clear:both;}
+.modal-footer .btn+.btn{margin-left:5px;margin-bottom:0;}
+.modal-footer .btn-group .btn+.btn{margin-left:-1px;}
+.modal-footer .btn-block+.btn-block{margin-left:0;}
+.dropup,.dropdown{position:relative;}
+.dropdown-toggle{*margin-bottom:-3px;}
+.dropdown-toggle:active,.open .dropdown-toggle{outline:0;}
+.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000000;border-right:4px solid transparent;border-left:4px solid transparent;content:"";}
+.dropdown .caret{margin-top:8px;margin-left:2px;}
+.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#ffffff;border:1px solid #ccc;border:1px solid rgba(0, 0, 0, 0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);-moz-box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box;}.dropdown-menu.pull-right{right:0;left:auto;}
+.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #ffffff;}
+.dropdown-menu>li>a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333333;white-space:nowrap;}
+.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus,.dropdown-submenu:hover>a,.dropdown-submenu:focus>a{text-decoration:none;color:#ffffff;background-color:#0081c2;background-image:-moz-linear-gradient(top, #0088cc, #0077b3);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3));background-image:-webkit-linear-gradient(top, #0088cc, #0077b3);background-image:-o-linear-gradient(top, #0088cc, #0077b3);background-image:linear-gradient(to bottom, #0088cc, #0077b3);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0);}
+.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{color:#ffffff;text-decoration:none;outline:0;background-color:#0081c2;background-image:-moz-linear-gradient(top, #0088cc, #0077b3);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3));background-image:-webkit-linear-gradient(top, #0088cc, #0077b3);background-image:-o-linear-gradient(top, #0088cc, #0077b3);background-image:linear-gradient(to bottom, #0088cc, #0077b3);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0);}
+.dropdown-menu>.disabled>a,.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{color:#999999;}
+.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{text-decoration:none;background-color:transparent;background-image:none;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);cursor:default;}
+.open{*z-index:1000;}.open>.dropdown-menu{display:block;}
+.dropdown-backdrop{position:fixed;left:0;right:0;bottom:0;top:0;z-index:990;}
+.pull-right>.dropdown-menu{right:0;left:auto;}
+.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000000;content:"";}
+.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px;}
+.dropdown-submenu{position:relative;}
+.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px;}
+.dropdown-submenu:hover>.dropdown-menu{display:block;}
+.dropup .dropdown-submenu>.dropdown-menu{top:auto;bottom:0;margin-top:0;margin-bottom:-2px;-webkit-border-radius:5px 5px 5px 0;-moz-border-radius:5px 5px 5px 0;border-radius:5px 5px 5px 0;}
+.dropdown-submenu>a:after{display:block;content:" ";float:right;width:0;height:0;border-color:transparent;border-style:solid;border-width:5px 0 5px 5px;border-left-color:#cccccc;margin-top:5px;margin-right:-10px;}
+.dropdown-submenu:hover>a:after{border-left-color:#ffffff;}
+.dropdown-submenu.pull-left{float:none;}.dropdown-submenu.pull-left>.dropdown-menu{left:-100%;margin-left:10px;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px;}
+.dropdown .dropdown-menu .nav-header{padding-left:20px;padding-right:20px;}
+.typeahead{z-index:1051;margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}
+.accordion{margin-bottom:20px;}
+.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}
+.accordion-heading{border-bottom:0;}
+.accordion-heading .accordion-toggle{display:block;padding:8px 15px;}
+.accordion-toggle{cursor:pointer;}
+.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5;}
+.carousel{position:relative;margin-bottom:20px;line-height:1;}
+.carousel-inner{overflow:hidden;width:100%;position:relative;}
+.carousel-inner>.item{display:none;position:relative;-webkit-transition:0.6s ease-in-out left;-moz-transition:0.6s ease-in-out left;-o-transition:0.6s ease-in-out left;transition:0.6s ease-in-out left;}.carousel-inner>.item>img,.carousel-inner>.item>a>img{display:block;line-height:1;}
+.carousel-inner>.active,.carousel-inner>.next,.carousel-inner>.prev{display:block;}
+.carousel-inner>.active{left:0;}
+.carousel-inner>.next,.carousel-inner>.prev{position:absolute;top:0;width:100%;}
+.carousel-inner>.next{left:100%;}
+.carousel-inner>.prev{left:-100%;}
+.carousel-inner>.next.left,.carousel-inner>.prev.right{left:0;}
+.carousel-inner>.active.left{left:-100%;}
+.carousel-inner>.active.right{left:100%;}
+.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#ffffff;text-align:center;background:#222222;border:3px solid #ffffff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:0.5;filter:alpha(opacity=50);}.carousel-control.right{left:auto;right:15px;}
+.carousel-control:hover,.carousel-control:focus{color:#ffffff;text-decoration:none;opacity:0.9;filter:alpha(opacity=90);}
+.carousel-indicators{position:absolute;top:15px;right:15px;z-index:5;margin:0;list-style:none;}.carousel-indicators li{display:block;float:left;width:10px;height:10px;margin-left:5px;text-indent:-999px;background-color:#ccc;background-color:rgba(255, 255, 255, 0.25);border-radius:5px;}
+.carousel-indicators .active{background-color:#fff;}
+.carousel-caption{position:absolute;left:0;right:0;bottom:0;padding:15px;background:#333333;background:rgba(0, 0, 0, 0.75);}
+.carousel-caption h4,.carousel-caption p{color:#ffffff;line-height:20px;}
+.carousel-caption h4{margin:0 0 5px;}
+.carousel-caption p{margin-bottom:0;}
+.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.05);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.05);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.05);}.well blockquote{border-color:#ddd;border-color:rgba(0, 0, 0, 0.15);}
+.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;}
+.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;}
+.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000000;text-shadow:0 1px 0 #ffffff;opacity:0.2;filter:alpha(opacity=20);}.close:hover,.close:focus{color:#000000;text-decoration:none;cursor:pointer;opacity:0.4;filter:alpha(opacity=40);}
+button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none;}
+.pull-right{float:right;}
+.pull-left{float:left;}
+.hide{display:none;}
+.show{display:block;}
+.invisible{visibility:hidden;}
+.affix{position:fixed;}
+.fade{opacity:0;-webkit-transition:opacity 0.15s linear;-moz-transition:opacity 0.15s linear;-o-transition:opacity 0.15s linear;transition:opacity 0.15s linear;}.fade.in{opacity:1;}
+.collapse{position:relative;height:0;overflow:hidden;-webkit-transition:height 0.35s ease;-moz-transition:height 0.35s ease;-o-transition:height 0.35s ease;transition:height 0.35s ease;}.collapse.in{height:auto;}
+@-ms-viewport{width:device-width;}.hidden{display:none;visibility:hidden;}
+.visible-phone{display:none !important;}
+.visible-tablet{display:none !important;}
+.hidden-desktop{display:none !important;}
+.visible-desktop{display:inherit !important;}
+@media (min-width:768px) and (max-width:979px){.hidden-desktop{display:inherit !important;} .visible-desktop{display:none !important ;} .visible-tablet{display:inherit !important;} .hidden-tablet{display:none !important;}}@media (max-width:767px){.hidden-desktop{display:inherit !important;} .visible-desktop{display:none !important;} .visible-phone{display:inherit !important;} .hidden-phone{display:none !important;}}.visible-print{display:none !important;}
+@media print{.visible-print{display:inherit !important;} .hidden-print{display:none !important;}}@media (max-width:767px){body{padding-left:20px;padding-right:20px;} .navbar-fixed-top,.navbar-fixed-bottom,.navbar-static-top{margin-left:-20px;margin-right:-20px;} .container-fluid{padding:0;} .dl-horizontal dt{float:none;clear:none;width:auto;text-align:left;} .dl-horizontal dd{margin-left:0;} .container{width:auto;} .row-fluid{width:100%;} .row,.thumbnails{margin-left:0;} .thumbnails>li{float:none;margin-left:0;} [class*="span"],.uneditable-input[class*="span"],.row-fluid [class*="span"]{float:none;display:block;width:100%;margin-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;} .span12,.row-fluid .span12{width:100%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;} .row-fluid [class*="offset"]:first-child{margin-left:0;} .input-large,.input-xlarge,.input-xxlarge,input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;} .input-prepend input,.input-append input,.input-prepend input[class*="span"],.input-append input[class*="span"]{display:inline-block;width:auto;} .controls-row [class*="span"]+[class*="span"]{margin-left:0;} .modal{position:fixed;top:20px;left:20px;right:20px;width:auto;margin:0;}.modal.fade{top:-100px;} .modal.fade.in{top:20px;}}@media (max-width:480px){.nav-collapse{-webkit-transform:translate3d(0, 0, 0);} .page-header h1 small{display:block;line-height:20px;} input[type="checkbox"],input[type="radio"]{border:1px solid #ccc;} .form-horizontal .control-label{float:none;width:auto;padding-top:0;text-align:left;} .form-horizontal .controls{margin-left:0;} .form-horizontal .control-list{padding-top:0;} .form-horizontal .form-actions{padding-left:10px;padding-right:10px;} .media .pull-left,.media .pull-right{float:none;display:block;margin-bottom:10px;} .media-object{margin-right:0;margin-left:0;} .modal{top:10px;left:10px;right:10px;} .modal-header .close{padding:10px;margin:-10px;} .carousel-caption{position:static;}}@media (min-width:768px) and (max-width:979px){.row{margin-left:-20px;*zoom:1;}.row:before,.row:after{display:table;content:"";line-height:0;} .row:after{clear:both;} [class*="span"]{float:left;min-height:1px;margin-left:20px;} .container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:724px;} .span12{width:724px;} .span11{width:662px;} .span10{width:600px;} .span9{width:538px;} .span8{width:476px;} .span7{width:414px;} .span6{width:352px;} .span5{width:290px;} .span4{width:228px;} .span3{width:166px;} .span2{width:104px;} .span1{width:42px;} .offset12{margin-left:764px;} .offset11{margin-left:702px;} .offset10{margin-left:640px;} .offset9{margin-left:578px;} .offset8{margin-left:516px;} .offset7{margin-left:454px;} .offset6{margin-left:392px;} .offset5{margin-left:330px;} .offset4{margin-left:268px;} .offset3{margin-left:206px;} .offset2{margin-left:144px;} .offset1{margin-left:82px;} .row-fluid{width:100%;*zoom:1;}.row-fluid:before,.row-fluid:after{display:table;content:"";line-height:0;} .row-fluid:after{clear:both;} .row-fluid [class*="span"]{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;float:left;margin-left:2.7624309392265194%;*margin-left:2.709239449864817%;} .row-fluid [class*="span"]:first-child{margin-left:0;} .row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.7624309392265194%;} .row-fluid .span12{width:100%;*width:99.94680851063829%;} .row-fluid .span11{width:91.43646408839778%;*width:91.38327259903608%;} .row-fluid .span10{width:82.87292817679558%;*width:82.81973668743387%;} .row-fluid .span9{width:74.30939226519337%;*width:74.25620077583166%;} .row-fluid .span8{width:65.74585635359117%;*width:65.69266486422946%;} .row-fluid .span7{width:57.18232044198895%;*width:57.12912895262725%;} .row-fluid .span6{width:48.61878453038674%;*width:48.56559304102504%;} .row-fluid .span5{width:40.05524861878453%;*width:40.00205712942283%;} .row-fluid .span4{width:31.491712707182323%;*width:31.43852121782062%;} .row-fluid .span3{width:22.92817679558011%;*width:22.87498530621841%;} .row-fluid .span2{width:14.3646408839779%;*width:14.311449394616199%;} .row-fluid .span1{width:5.801104972375691%;*width:5.747913483013988%;} .row-fluid .offset12{margin-left:105.52486187845304%;*margin-left:105.41847889972962%;} .row-fluid .offset12:first-child{margin-left:102.76243093922652%;*margin-left:102.6560479605031%;} .row-fluid .offset11{margin-left:96.96132596685082%;*margin-left:96.8549429881274%;} .row-fluid .offset11:first-child{margin-left:94.1988950276243%;*margin-left:94.09251204890089%;} .row-fluid .offset10{margin-left:88.39779005524862%;*margin-left:88.2914070765252%;} .row-fluid .offset10:first-child{margin-left:85.6353591160221%;*margin-left:85.52897613729868%;} .row-fluid .offset9{margin-left:79.8342541436464%;*margin-left:79.72787116492299%;} .row-fluid .offset9:first-child{margin-left:77.07182320441989%;*margin-left:76.96544022569647%;} .row-fluid .offset8{margin-left:71.2707182320442%;*margin-left:71.16433525332079%;} .row-fluid .offset8:first-child{margin-left:68.50828729281768%;*margin-left:68.40190431409427%;} .row-fluid .offset7{margin-left:62.70718232044199%;*margin-left:62.600799341718584%;} .row-fluid .offset7:first-child{margin-left:59.94475138121547%;*margin-left:59.838368402492065%;} .row-fluid .offset6{margin-left:54.14364640883978%;*margin-left:54.037263430116376%;} .row-fluid .offset6:first-child{margin-left:51.38121546961326%;*margin-left:51.27483249088986%;} .row-fluid .offset5{margin-left:45.58011049723757%;*margin-left:45.47372751851417%;} .row-fluid .offset5:first-child{margin-left:42.81767955801105%;*margin-left:42.71129657928765%;} .row-fluid .offset4{margin-left:37.01657458563536%;*margin-left:36.91019160691196%;} .row-fluid .offset4:first-child{margin-left:34.25414364640884%;*margin-left:34.14776066768544%;} .row-fluid .offset3{margin-left:28.45303867403315%;*margin-left:28.346655695309746%;} .row-fluid .offset3:first-child{margin-left:25.69060773480663%;*margin-left:25.584224756083227%;} .row-fluid .offset2{margin-left:19.88950276243094%;*margin-left:19.783119783707537%;} .row-fluid .offset2:first-child{margin-left:17.12707182320442%;*margin-left:17.02068884448102%;} .row-fluid .offset1{margin-left:11.32596685082873%;*margin-left:11.219583872105325%;} .row-fluid .offset1:first-child{margin-left:8.56353591160221%;*margin-left:8.457152932878806%;} input,textarea,.uneditable-input{margin-left:0;} .controls-row [class*="span"]+[class*="span"]{margin-left:20px;} input.span12,textarea.span12,.uneditable-input.span12{width:710px;} input.span11,textarea.span11,.uneditable-input.span11{width:648px;} input.span10,textarea.span10,.uneditable-input.span10{width:586px;} input.span9,textarea.span9,.uneditable-input.span9{width:524px;} input.span8,textarea.span8,.uneditable-input.span8{width:462px;} input.span7,textarea.span7,.uneditable-input.span7{width:400px;} input.span6,textarea.span6,.uneditable-input.span6{width:338px;} input.span5,textarea.span5,.uneditable-input.span5{width:276px;} input.span4,textarea.span4,.uneditable-input.span4{width:214px;} input.span3,textarea.span3,.uneditable-input.span3{width:152px;} input.span2,textarea.span2,.uneditable-input.span2{width:90px;} input.span1,textarea.span1,.uneditable-input.span1{width:28px;}}@media (min-width:1200px){.row{margin-left:-30px;*zoom:1;}.row:before,.row:after{display:table;content:"";line-height:0;} .row:after{clear:both;} [class*="span"]{float:left;min-height:1px;margin-left:30px;} .container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:1170px;} .span12{width:1170px;} .span11{width:1070px;} .span10{width:970px;} .span9{width:870px;} .span8{width:770px;} .span7{width:670px;} .span6{width:570px;} .span5{width:470px;} .span4{width:370px;} .span3{width:270px;} .span2{width:170px;} .span1{width:70px;} .offset12{margin-left:1230px;} .offset11{margin-left:1130px;} .offset10{margin-left:1030px;} .offset9{margin-left:930px;} .offset8{margin-left:830px;} .offset7{margin-left:730px;} .offset6{margin-left:630px;} .offset5{margin-left:530px;} .offset4{margin-left:430px;} .offset3{margin-left:330px;} .offset2{margin-left:230px;} .offset1{margin-left:130px;} .row-fluid{width:100%;*zoom:1;}.row-fluid:before,.row-fluid:after{display:table;content:"";line-height:0;} .row-fluid:after{clear:both;} .row-fluid [class*="span"]{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;float:left;margin-left:2.564102564102564%;*margin-left:2.5109110747408616%;} .row-fluid [class*="span"]:first-child{margin-left:0;} .row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.564102564102564%;} .row-fluid .span12{width:100%;*width:99.94680851063829%;} .row-fluid .span11{width:91.45299145299145%;*width:91.39979996362975%;} .row-fluid .span10{width:82.90598290598291%;*width:82.8527914166212%;} .row-fluid .span9{width:74.35897435897436%;*width:74.30578286961266%;} .row-fluid .span8{width:65.81196581196582%;*width:65.75877432260411%;} .row-fluid .span7{width:57.26495726495726%;*width:57.21176577559556%;} .row-fluid .span6{width:48.717948717948715%;*width:48.664757228587014%;} .row-fluid .span5{width:40.17094017094017%;*width:40.11774868157847%;} .row-fluid .span4{width:31.623931623931625%;*width:31.570740134569924%;} .row-fluid .span3{width:23.076923076923077%;*width:23.023731587561375%;} .row-fluid .span2{width:14.52991452991453%;*width:14.476723040552828%;} .row-fluid .span1{width:5.982905982905983%;*width:5.929714493544281%;} .row-fluid .offset12{margin-left:105.12820512820512%;*margin-left:105.02182214948171%;} .row-fluid .offset12:first-child{margin-left:102.56410256410257%;*margin-left:102.45771958537915%;} .row-fluid .offset11{margin-left:96.58119658119658%;*margin-left:96.47481360247316%;} .row-fluid .offset11:first-child{margin-left:94.01709401709402%;*margin-left:93.91071103837061%;} .row-fluid .offset10{margin-left:88.03418803418803%;*margin-left:87.92780505546462%;} .row-fluid .offset10:first-child{margin-left:85.47008547008548%;*margin-left:85.36370249136206%;} .row-fluid .offset9{margin-left:79.48717948717949%;*margin-left:79.38079650845607%;} .row-fluid .offset9:first-child{margin-left:76.92307692307693%;*margin-left:76.81669394435352%;} .row-fluid .offset8{margin-left:70.94017094017094%;*margin-left:70.83378796144753%;} .row-fluid .offset8:first-child{margin-left:68.37606837606839%;*margin-left:68.26968539734497%;} .row-fluid .offset7{margin-left:62.393162393162385%;*margin-left:62.28677941443899%;} .row-fluid .offset7:first-child{margin-left:59.82905982905982%;*margin-left:59.72267685033642%;} .row-fluid .offset6{margin-left:53.84615384615384%;*margin-left:53.739770867430444%;} .row-fluid .offset6:first-child{margin-left:51.28205128205128%;*margin-left:51.175668303327875%;} .row-fluid .offset5{margin-left:45.299145299145295%;*margin-left:45.1927623204219%;} .row-fluid .offset5:first-child{margin-left:42.73504273504273%;*margin-left:42.62865975631933%;} .row-fluid .offset4{margin-left:36.75213675213675%;*margin-left:36.645753773413354%;} .row-fluid .offset4:first-child{margin-left:34.18803418803419%;*margin-left:34.081651209310785%;} .row-fluid .offset3{margin-left:28.205128205128204%;*margin-left:28.0987452264048%;} .row-fluid .offset3:first-child{margin-left:25.641025641025642%;*margin-left:25.53464266230224%;} .row-fluid .offset2{margin-left:19.65811965811966%;*margin-left:19.551736679396257%;} .row-fluid .offset2:first-child{margin-left:17.094017094017094%;*margin-left:16.98763411529369%;} .row-fluid .offset1{margin-left:11.11111111111111%;*margin-left:11.004728132387708%;} .row-fluid .offset1:first-child{margin-left:8.547008547008547%;*margin-left:8.440625568285142%;} input,textarea,.uneditable-input{margin-left:0;} .controls-row [class*="span"]+[class*="span"]{margin-left:30px;} input.span12,textarea.span12,.uneditable-input.span12{width:1156px;} input.span11,textarea.span11,.uneditable-input.span11{width:1056px;} input.span10,textarea.span10,.uneditable-input.span10{width:956px;} input.span9,textarea.span9,.uneditable-input.span9{width:856px;} input.span8,textarea.span8,.uneditable-input.span8{width:756px;} input.span7,textarea.span7,.uneditable-input.span7{width:656px;} input.span6,textarea.span6,.uneditable-input.span6{width:556px;} input.span5,textarea.span5,.uneditable-input.span5{width:456px;} input.span4,textarea.span4,.uneditable-input.span4{width:356px;} input.span3,textarea.span3,.uneditable-input.span3{width:256px;} input.span2,textarea.span2,.uneditable-input.span2{width:156px;} input.span1,textarea.span1,.uneditable-input.span1{width:56px;} .thumbnails{margin-left:-30px;} .thumbnails>li{margin-left:30px;} .row-fluid .thumbnails{margin-left:0;}}@media (max-width:979px){body{padding-top:0;} .navbar-fixed-top,.navbar-fixed-bottom{position:static;} .navbar-fixed-top{margin-bottom:20px;} .navbar-fixed-bottom{margin-top:20px;} .navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding:5px;} .navbar .container{width:auto;padding:0;} .navbar .brand{padding-left:10px;padding-right:10px;margin:0 0 0 -5px;} .nav-collapse{clear:both;} .nav-collapse .nav{float:none;margin:0 0 10px;} .nav-collapse .nav>li{float:none;} .nav-collapse .nav>li>a{margin-bottom:2px;} .nav-collapse .nav>.divider-vertical{display:none;} .nav-collapse .nav .nav-header{color:#777777;text-shadow:none;} .nav-collapse .nav>li>a,.nav-collapse .dropdown-menu a{padding:9px 15px;font-weight:bold;color:#777777;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;} .nav-collapse .btn{padding:4px 10px 4px;font-weight:normal;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;} .nav-collapse .dropdown-menu li+li a{margin-bottom:2px;} .nav-collapse .nav>li>a:hover,.nav-collapse .nav>li>a:focus,.nav-collapse .dropdown-menu a:hover,.nav-collapse .dropdown-menu a:focus{background-color:#f2f2f2;} .navbar-inverse .nav-collapse .nav>li>a,.navbar-inverse .nav-collapse .dropdown-menu a{color:#999999;} .navbar-inverse .nav-collapse .nav>li>a:hover,.navbar-inverse .nav-collapse .nav>li>a:focus,.navbar-inverse .nav-collapse .dropdown-menu a:hover,.navbar-inverse .nav-collapse .dropdown-menu a:focus{background-color:#111111;} .nav-collapse.in .btn-group{margin-top:5px;padding:0;} .nav-collapse .dropdown-menu{position:static;top:auto;left:auto;float:none;display:none;max-width:none;margin:0 15px;padding:0;background-color:transparent;border:none;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none;} .nav-collapse .open>.dropdown-menu{display:block;} .nav-collapse .dropdown-menu:before,.nav-collapse .dropdown-menu:after{display:none;} .nav-collapse .dropdown-menu .divider{display:none;} .nav-collapse .nav>li>.dropdown-menu:before,.nav-collapse .nav>li>.dropdown-menu:after{display:none;} .nav-collapse .navbar-form,.nav-collapse .navbar-search{float:none;padding:10px 15px;margin:10px 0;border-top:1px solid #f2f2f2;border-bottom:1px solid #f2f2f2;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.1);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.1);box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.1);} .navbar-inverse .nav-collapse .navbar-form,.navbar-inverse .nav-collapse .navbar-search{border-top-color:#111111;border-bottom-color:#111111;} .navbar .nav-collapse .nav.pull-right{float:none;margin-left:0;} .nav-collapse,.nav-collapse.collapse{overflow:hidden;height:0;} .navbar .btn-navbar{display:block;} .navbar-static .navbar-inner{padding-left:10px;padding-right:10px;}}@media (min-width:980px){.nav-collapse.collapse{height:auto !important;overflow:visible !important;}}
diff --git a/asterix-app/src/main/resources/webui/static/css/style.css b/asterix-app/src/main/resources/webui/static/css/style.css
index b8688db..a3e411c 100644
--- a/asterix-app/src/main/resources/webui/static/css/style.css
+++ b/asterix-app/src/main/resources/webui/static/css/style.css
@@ -1,6 +1,6 @@
 body {
     background: none repeat scroll 0 0 white;
-    color: #222222;
+    color: black;
     font-family: 'Bitter';
     font-size: 14px;
     line-height: 17px;
@@ -8,13 +8,14 @@
 }
 
 .content {
-    margin-top: 70px;
+    margin-top: 36px;
 }
 
 label.query, label.result {
     font-size: 24px;
     padding-bottom: 10px;
     font-weight: bold;
+    color : #17265a;
 }
 
 div.host {
@@ -34,10 +35,14 @@
 div.right {
 }
 
-button.btn {
-    clear: both;
-    float: left;
-    margin: 20px 0 0 10px;;
+#run-btn {
+}
+
+#select-toggle-holder {
+    border: 1px black;
+    -webkit-border-radius: 5px;
+    -moz-border-radius: 5px;
+    border-radius: 5px;
 }
 
 textarea.query {
@@ -59,12 +64,18 @@
     padding-top: 10px;
 }
 
+#opts {
+    margin-right: 4px;
+}
+
 input[type=text] {
     height: 20px;
 }
 
 pre {
+    overflow-x : auto;
     overflow: auto;
+    overflow-wrap: normal;
     white-space: pre;
 }
 
@@ -73,6 +84,27 @@
     margin-top: 2px;
     padding-bottom: 10px;
     font-weight: bold;
+    color : #17265a;
+}
+
+div.output label.heading.error {
+    color: #E03809; 
+}
+
+div.output h4 {
+    color : #17265a;
+}
+
+a.accordion-toggle {
+    color : #17265a;
+}
+
+.accordion-inner {
+    padding : 0 0 0 0;
+}
+
+#errorblock .accordion-group .accordion-heading a.accordion-toggle {
+    color: #E03809;
 }
 
 div.output .message {
@@ -85,22 +117,7 @@
 }
 
 div.output .message pre.error {
-    -webkit-box-sizing: border-box;
-    -moz-box-sizing: border-box;
-    -ms-box-sizing: border-box;
-    box-sizing: border-box;
-    border-color: rgba(82, 168, 236, 0.8);
-    outline: 0;
-    outline: thin dotted 9;
-
-    -webkit-box-shadow: inset 0 1px 1px rgba(250, 0, 0, 0.075), 0 0 8px rgba(255, 0, 0, 0.8);
-    -moz-box-shadow: inset 0 1px 1px rgba(250, 0, 0, 0.075), 0 0 8px rgba(255, 0, 0, 1.0);
-    box-shadow: inset 0 1px 1px rgba(250, 0, 0, 0.075), 0 0 8px rgba(255, 0, 0, 1.0);
-    color: #000;
-    resize: none;
-    border: 1px solid #eee;
-    margin-top: 7px;
-    padding: 20px 20px 20px 20px;
+   border: 0;
 }
 
 .footer {
@@ -137,3 +154,60 @@
     padding-right: 20px;
     float: right;
 }
+
+.navbar .brand {
+    width: 160px;
+    height: 40px;
+    overflow: visible;
+    padding-top: 0;
+    padding-bottom: 0;
+}
+
+/*.navbar .nav > li > a {
+    color: #17265a;
+}*/
+
+.extarget {
+    margin-left: 4px;
+    padding-bottom: 2px;
+}
+
+.btn-custom-darken.active {
+  color: rgba(255, 255, 255, 0.75);
+}
+.btn-custom-darken {
+  color: #ffffff;
+  text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);
+  background-color: #233887;
+  background-image: -moz-linear-gradient(top, #273f97, #1d2e6e);
+  background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#273f97), to(#1d2e6e));
+  background-image: -webkit-linear-gradient(top, #273f97, #1d2e6e);
+  background-image: -o-linear-gradient(top, #273f97, #1d2e6e);
+  background-image: linear-gradient(to bottom, #273f97, #1d2e6e);
+  background-repeat: repeat-x;
+  filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff273f97', endColorstr='#ff1d2e6e', GradientType=0);
+  border-color: #1d2e6e #1d2e6e #0d1532;
+  border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);
+  *background-color: #1d2e6e;
+  /* Darken IE7 buttons by default so they stand out more given they won't have borders */
+
+  filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);
+}
+.btn-custom-darken:hover,
+.btn-custom-darken:focus,
+.btn-custom-darken:active,
+.btn-custom-darken.active,
+.btn-custom-darken.disabled,
+.btn-custom-darken[disabled] {
+  color: #ffffff;
+  background-color: #1d2e6e;
+  *background-color: #17265a;
+}
+.btn-custom-darken:active,
+.btn-custom-darken.active {
+  background-color: #121d46 ;
+}
+
+.span6 {
+    padding: 24px;
+}
diff --git a/asterix-app/src/main/resources/webui/static/img/finalasterixlogo.jpg b/asterix-app/src/main/resources/webui/static/img/finalasterixlogo.jpg
new file mode 100644
index 0000000..0ab3727
--- /dev/null
+++ b/asterix-app/src/main/resources/webui/static/img/finalasterixlogo.jpg
Binary files differ
diff --git a/asterix-app/src/main/resources/webui/static/img/finalasterixlogo.png b/asterix-app/src/main/resources/webui/static/img/finalasterixlogo.png
new file mode 100644
index 0000000..10ce431
--- /dev/null
+++ b/asterix-app/src/main/resources/webui/static/img/finalasterixlogo.png
Binary files differ
diff --git a/asterix-app/src/main/resources/webui/static/img/targetlink.png b/asterix-app/src/main/resources/webui/static/img/targetlink.png
new file mode 100644
index 0000000..68ceb60
--- /dev/null
+++ b/asterix-app/src/main/resources/webui/static/img/targetlink.png
Binary files differ
diff --git a/asterix-app/src/main/resources/webui/static/js/bootstrap.min.js b/asterix-app/src/main/resources/webui/static/js/bootstrap.min.js
index 95c5ac5..319a85d 100644
--- a/asterix-app/src/main/resources/webui/static/js/bootstrap.min.js
+++ b/asterix-app/src/main/resources/webui/static/js/bootstrap.min.js
@@ -1,6 +1,7 @@
-/*!
+/**
 * Bootstrap.js by @fat & @mdo
+* plugins: bootstrap-transition.js, bootstrap-modal.js, bootstrap-dropdown.js, bootstrap-scrollspy.js, bootstrap-tab.js, bootstrap-tooltip.js, bootstrap-popover.js, bootstrap-affix.js, bootstrap-alert.js, bootstrap-button.js, bootstrap-collapse.js, bootstrap-carousel.js, bootstrap-typeahead.js
 * Copyright 2012 Twitter, Inc.
 * http://www.apache.org/licenses/LICENSE-2.0.txt
 */
-!function(e){"use strict";e(function(){e.support.transition=function(){var e=function(){var e=document.createElement("bootstrap"),t={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"},n;for(n in t)if(e.style[n]!==undefined)return t[n]}();return e&&{end:e}}()})}(window.jQuery),!function(e){"use strict";var t='[data-dismiss="alert"]',n=function(n){e(n).on("click",t,this.close)};n.prototype.close=function(t){function s(){i.trigger("closed").remove()}var n=e(this),r=n.attr("data-target"),i;r||(r=n.attr("href"),r=r&&r.replace(/.*(?=#[^\s]*$)/,"")),i=e(r),t&&t.preventDefault(),i.length||(i=n.hasClass("alert")?n:n.parent()),i.trigger(t=e.Event("close"));if(t.isDefaultPrevented())return;i.removeClass("in"),e.support.transition&&i.hasClass("fade")?i.on(e.support.transition.end,s):s()};var r=e.fn.alert;e.fn.alert=function(t){return this.each(function(){var r=e(this),i=r.data("alert");i||r.data("alert",i=new n(this)),typeof t=="string"&&i[t].call(r)})},e.fn.alert.Constructor=n,e.fn.alert.noConflict=function(){return e.fn.alert=r,this},e(document).on("click.alert.data-api",t,n.prototype.close)}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.options=e.extend({},e.fn.button.defaults,n)};t.prototype.setState=function(e){var t="disabled",n=this.$element,r=n.data(),i=n.is("input")?"val":"html";e+="Text",r.resetText||n.data("resetText",n[i]()),n[i](r[e]||this.options[e]),setTimeout(function(){e=="loadingText"?n.addClass(t).attr(t,t):n.removeClass(t).removeAttr(t)},0)},t.prototype.toggle=function(){var e=this.$element.closest('[data-toggle="buttons-radio"]');e&&e.find(".active").removeClass("active"),this.$element.toggleClass("active")};var n=e.fn.button;e.fn.button=function(n){return this.each(function(){var r=e(this),i=r.data("button"),s=typeof n=="object"&&n;i||r.data("button",i=new t(this,s)),n=="toggle"?i.toggle():n&&i.setState(n)})},e.fn.button.defaults={loadingText:"loading..."},e.fn.button.Constructor=t,e.fn.button.noConflict=function(){return e.fn.button=n,this},e(document).on("click.button.data-api","[data-toggle^=button]",function(t){var n=e(t.target);n.hasClass("btn")||(n=n.closest(".btn")),n.button("toggle")})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.$indicators=this.$element.find(".carousel-indicators"),this.options=n,this.options.pause=="hover"&&this.$element.on("mouseenter",e.proxy(this.pause,this)).on("mouseleave",e.proxy(this.cycle,this))};t.prototype={cycle:function(t){return t||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(e.proxy(this.next,this),this.options.interval)),this},getActiveIndex:function(){return this.$active=this.$element.find(".item.active"),this.$items=this.$active.parent().children(),this.$items.index(this.$active)},to:function(t){var n=this.getActiveIndex(),r=this;if(t>this.$items.length-1||t<0)return;return this.sliding?this.$element.one("slid",function(){r.to(t)}):n==t?this.pause().cycle():this.slide(t>n?"next":"prev",e(this.$items[t]))},pause:function(t){return t||(this.paused=!0),this.$element.find(".next, .prev").length&&e.support.transition.end&&(this.$element.trigger(e.support.transition.end),this.cycle(!0)),clearInterval(this.interval),this.interval=null,this},next:function(){if(this.sliding)return;return this.slide("next")},prev:function(){if(this.sliding)return;return this.slide("prev")},slide:function(t,n){var r=this.$element.find(".item.active"),i=n||r[t](),s=this.interval,o=t=="next"?"left":"right",u=t=="next"?"first":"last",a=this,f;this.sliding=!0,s&&this.pause(),i=i.length?i:this.$element.find(".item")[u](),f=e.Event("slide",{relatedTarget:i[0],direction:o});if(i.hasClass("active"))return;this.$indicators.length&&(this.$indicators.find(".active").removeClass("active"),this.$element.one("slid",function(){var t=e(a.$indicators.children()[a.getActiveIndex()]);t&&t.addClass("active")}));if(e.support.transition&&this.$element.hasClass("slide")){this.$element.trigger(f);if(f.isDefaultPrevented())return;i.addClass(t),i[0].offsetWidth,r.addClass(o),i.addClass(o),this.$element.one(e.support.transition.end,function(){i.removeClass([t,o].join(" ")).addClass("active"),r.removeClass(["active",o].join(" ")),a.sliding=!1,setTimeout(function(){a.$element.trigger("slid")},0)})}else{this.$element.trigger(f);if(f.isDefaultPrevented())return;r.removeClass("active"),i.addClass("active"),this.sliding=!1,this.$element.trigger("slid")}return s&&this.cycle(),this}};var n=e.fn.carousel;e.fn.carousel=function(n){return this.each(function(){var r=e(this),i=r.data("carousel"),s=e.extend({},e.fn.carousel.defaults,typeof n=="object"&&n),o=typeof n=="string"?n:s.slide;i||r.data("carousel",i=new t(this,s)),typeof n=="number"?i.to(n):o?i[o]():s.interval&&i.pause().cycle()})},e.fn.carousel.defaults={interval:5e3,pause:"hover"},e.fn.carousel.Constructor=t,e.fn.carousel.noConflict=function(){return e.fn.carousel=n,this},e(document).on("click.carousel.data-api","[data-slide], [data-slide-to]",function(t){var n=e(this),r,i=e(n.attr("data-target")||(r=n.attr("href"))&&r.replace(/.*(?=#[^\s]+$)/,"")),s=e.extend({},i.data(),n.data()),o;i.carousel(s),(o=n.attr("data-slide-to"))&&i.data("carousel").pause().to(o).cycle(),t.preventDefault()})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.options=e.extend({},e.fn.collapse.defaults,n),this.options.parent&&(this.$parent=e(this.options.parent)),this.options.toggle&&this.toggle()};t.prototype={constructor:t,dimension:function(){var e=this.$element.hasClass("width");return e?"width":"height"},show:function(){var t,n,r,i;if(this.transitioning||this.$element.hasClass("in"))return;t=this.dimension(),n=e.camelCase(["scroll",t].join("-")),r=this.$parent&&this.$parent.find("> .accordion-group > .in");if(r&&r.length){i=r.data("collapse");if(i&&i.transitioning)return;r.collapse("hide"),i||r.data("collapse",null)}this.$element[t](0),this.transition("addClass",e.Event("show"),"shown"),e.support.transition&&this.$element[t](this.$element[0][n])},hide:function(){var t;if(this.transitioning||!this.$element.hasClass("in"))return;t=this.dimension(),this.reset(this.$element[t]()),this.transition("removeClass",e.Event("hide"),"hidden"),this.$element[t](0)},reset:function(e){var t=this.dimension();return this.$element.removeClass("collapse")[t](e||"auto")[0].offsetWidth,this.$element[e!==null?"addClass":"removeClass"]("collapse"),this},transition:function(t,n,r){var i=this,s=function(){n.type=="show"&&i.reset(),i.transitioning=0,i.$element.trigger(r)};this.$element.trigger(n);if(n.isDefaultPrevented())return;this.transitioning=1,this.$element[t]("in"),e.support.transition&&this.$element.hasClass("collapse")?this.$element.one(e.support.transition.end,s):s()},toggle:function(){this[this.$element.hasClass("in")?"hide":"show"]()}};var n=e.fn.collapse;e.fn.collapse=function(n){return this.each(function(){var r=e(this),i=r.data("collapse"),s=e.extend({},e.fn.collapse.defaults,r.data(),typeof n=="object"&&n);i||r.data("collapse",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.collapse.defaults={toggle:!0},e.fn.collapse.Constructor=t,e.fn.collapse.noConflict=function(){return e.fn.collapse=n,this},e(document).on("click.collapse.data-api","[data-toggle=collapse]",function(t){var n=e(this),r,i=n.attr("data-target")||t.preventDefault()||(r=n.attr("href"))&&r.replace(/.*(?=#[^\s]+$)/,""),s=e(i).data("collapse")?"toggle":n.data();n[e(i).hasClass("in")?"addClass":"removeClass"]("collapsed"),e(i).collapse(s)})}(window.jQuery),!function(e){"use strict";function r(){e(t).each(function(){i(e(this)).removeClass("open")})}function i(t){var n=t.attr("data-target"),r;n||(n=t.attr("href"),n=n&&/#/.test(n)&&n.replace(/.*(?=#[^\s]*$)/,"")),r=n&&e(n);if(!r||!r.length)r=t.parent();return r}var t="[data-toggle=dropdown]",n=function(t){var n=e(t).on("click.dropdown.data-api",this.toggle);e("html").on("click.dropdown.data-api",function(){n.parent().removeClass("open")})};n.prototype={constructor:n,toggle:function(t){var n=e(this),s,o;if(n.is(".disabled, :disabled"))return;return s=i(n),o=s.hasClass("open"),r(),o||s.toggleClass("open"),n.focus(),!1},keydown:function(n){var r,s,o,u,a,f;if(!/(38|40|27)/.test(n.keyCode))return;r=e(this),n.preventDefault(),n.stopPropagation();if(r.is(".disabled, :disabled"))return;u=i(r),a=u.hasClass("open");if(!a||a&&n.keyCode==27)return n.which==27&&u.find(t).focus(),r.click();s=e("[role=menu] li:not(.divider):visible a",u);if(!s.length)return;f=s.index(s.filter(":focus")),n.keyCode==38&&f>0&&f--,n.keyCode==40&&f<s.length-1&&f++,~f||(f=0),s.eq(f).focus()}};var s=e.fn.dropdown;e.fn.dropdown=function(t){return this.each(function(){var r=e(this),i=r.data("dropdown");i||r.data("dropdown",i=new n(this)),typeof t=="string"&&i[t].call(r)})},e.fn.dropdown.Constructor=n,e.fn.dropdown.noConflict=function(){return e.fn.dropdown=s,this},e(document).on("click.dropdown.data-api",r).on("click.dropdown.data-api",".dropdown form",function(e){e.stopPropagation()}).on("click.dropdown-menu",function(e){e.stopPropagation()}).on("click.dropdown.data-api",t,n.prototype.toggle).on("keydown.dropdown.data-api",t+", [role=menu]",n.prototype.keydown)}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.options=n,this.$element=e(t).delegate('[data-dismiss="modal"]',"click.dismiss.modal",e.proxy(this.hide,this)),this.options.remote&&this.$element.find(".modal-body").load(this.options.remote)};t.prototype={constructor:t,toggle:function(){return this[this.isShown?"hide":"show"]()},show:function(){var t=this,n=e.Event("show");this.$element.trigger(n);if(this.isShown||n.isDefaultPrevented())return;this.isShown=!0,this.escape(),this.backdrop(function(){var n=e.support.transition&&t.$element.hasClass("fade");t.$element.parent().length||t.$element.appendTo(document.body),t.$element.show(),n&&t.$element[0].offsetWidth,t.$element.addClass("in").attr("aria-hidden",!1),t.enforceFocus(),n?t.$element.one(e.support.transition.end,function(){t.$element.focus().trigger("shown")}):t.$element.focus().trigger("shown")})},hide:function(t){t&&t.preventDefault();var n=this;t=e.Event("hide"),this.$element.trigger(t);if(!this.isShown||t.isDefaultPrevented())return;this.isShown=!1,this.escape(),e(document).off("focusin.modal"),this.$element.removeClass("in").attr("aria-hidden",!0),e.support.transition&&this.$element.hasClass("fade")?this.hideWithTransition():this.hideModal()},enforceFocus:function(){var t=this;e(document).on("focusin.modal",function(e){t.$element[0]!==e.target&&!t.$element.has(e.target).length&&t.$element.focus()})},escape:function(){var e=this;this.isShown&&this.options.keyboard?this.$element.on("keyup.dismiss.modal",function(t){t.which==27&&e.hide()}):this.isShown||this.$element.off("keyup.dismiss.modal")},hideWithTransition:function(){var t=this,n=setTimeout(function(){t.$element.off(e.support.transition.end),t.hideModal()},500);this.$element.one(e.support.transition.end,function(){clearTimeout(n),t.hideModal()})},hideModal:function(){var e=this;this.$element.hide(),this.backdrop(function(){e.removeBackdrop(),e.$element.trigger("hidden")})},removeBackdrop:function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},backdrop:function(t){var n=this,r=this.$element.hasClass("fade")?"fade":"";if(this.isShown&&this.options.backdrop){var i=e.support.transition&&r;this.$backdrop=e('<div class="modal-backdrop '+r+'" />').appendTo(document.body),this.$backdrop.click(this.options.backdrop=="static"?e.proxy(this.$element[0].focus,this.$element[0]):e.proxy(this.hide,this)),i&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in");if(!t)return;i?this.$backdrop.one(e.support.transition.end,t):t()}else!this.isShown&&this.$backdrop?(this.$backdrop.removeClass("in"),e.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one(e.support.transition.end,t):t()):t&&t()}};var n=e.fn.modal;e.fn.modal=function(n){return this.each(function(){var r=e(this),i=r.data("modal"),s=e.extend({},e.fn.modal.defaults,r.data(),typeof n=="object"&&n);i||r.data("modal",i=new t(this,s)),typeof n=="string"?i[n]():s.show&&i.show()})},e.fn.modal.defaults={backdrop:!0,keyboard:!0,show:!0},e.fn.modal.Constructor=t,e.fn.modal.noConflict=function(){return e.fn.modal=n,this},e(document).on("click.modal.data-api",'[data-toggle="modal"]',function(t){var n=e(this),r=n.attr("href"),i=e(n.attr("data-target")||r&&r.replace(/.*(?=#[^\s]+$)/,"")),s=i.data("modal")?"toggle":e.extend({remote:!/#/.test(r)&&r},i.data(),n.data());t.preventDefault(),i.modal(s).one("hide",function(){n.focus()})})}(window.jQuery),!function(e){"use strict";var t=function(e,t){this.init("tooltip",e,t)};t.prototype={constructor:t,init:function(t,n,r){var i,s,o,u,a;this.type=t,this.$element=e(n),this.options=this.getOptions(r),this.enabled=!0,o=this.options.trigger.split(" ");for(a=o.length;a--;)u=o[a],u=="click"?this.$element.on("click."+this.type,this.options.selector,e.proxy(this.toggle,this)):u!="manual"&&(i=u=="hover"?"mouseenter":"focus",s=u=="hover"?"mouseleave":"blur",this.$element.on(i+"."+this.type,this.options.selector,e.proxy(this.enter,this)),this.$element.on(s+"."+this.type,this.options.selector,e.proxy(this.leave,this)));this.options.selector?this._options=e.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},getOptions:function(t){return t=e.extend({},e.fn[this.type].defaults,this.$element.data(),t),t.delay&&typeof t.delay=="number"&&(t.delay={show:t.delay,hide:t.delay}),t},enter:function(t){var n=e.fn[this.type].defaults,r={},i;this._options&&e.each(this._options,function(e,t){n[e]!=t&&(r[e]=t)},this),i=e(t.currentTarget)[this.type](r).data(this.type);if(!i.options.delay||!i.options.delay.show)return i.show();clearTimeout(this.timeout),i.hoverState="in",this.timeout=setTimeout(function(){i.hoverState=="in"&&i.show()},i.options.delay.show)},leave:function(t){var n=e(t.currentTarget)[this.type](this._options).data(this.type);this.timeout&&clearTimeout(this.timeout);if(!n.options.delay||!n.options.delay.hide)return n.hide();n.hoverState="out",this.timeout=setTimeout(function(){n.hoverState=="out"&&n.hide()},n.options.delay.hide)},show:function(){var t,n,r,i,s,o,u=e.Event("show");if(this.hasContent()&&this.enabled){this.$element.trigger(u);if(u.isDefaultPrevented())return;t=this.tip(),this.setContent(),this.options.animation&&t.addClass("fade"),s=typeof this.options.placement=="function"?this.options.placement.call(this,t[0],this.$element[0]):this.options.placement,t.detach().css({top:0,left:0,display:"block"}),this.options.container?t.appendTo(this.options.container):t.insertAfter(this.$element),n=this.getPosition(),r=t[0].offsetWidth,i=t[0].offsetHeight;switch(s){case"bottom":o={top:n.top+n.height,left:n.left+n.width/2-r/2};break;case"top":o={top:n.top-i,left:n.left+n.width/2-r/2};break;case"left":o={top:n.top+n.height/2-i/2,left:n.left-r};break;case"right":o={top:n.top+n.height/2-i/2,left:n.left+n.width}}this.applyPlacement(o,s),this.$element.trigger("shown")}},applyPlacement:function(e,t){var n=this.tip(),r=n[0].offsetWidth,i=n[0].offsetHeight,s,o,u,a;n.offset(e).addClass(t).addClass("in"),s=n[0].offsetWidth,o=n[0].offsetHeight,t=="top"&&o!=i&&(e.top=e.top+i-o,a=!0),t=="bottom"||t=="top"?(u=0,e.left<0&&(u=e.left*-2,e.left=0,n.offset(e),s=n[0].offsetWidth,o=n[0].offsetHeight),this.replaceArrow(u-r+s,s,"left")):this.replaceArrow(o-i,o,"top"),a&&n.offset(e)},replaceArrow:function(e,t,n){this.arrow().css(n,e?50*(1-e/t)+"%":"")},setContent:function(){var e=this.tip(),t=this.getTitle();e.find(".tooltip-inner")[this.options.html?"html":"text"](t),e.removeClass("fade in top bottom left right")},hide:function(){function i(){var t=setTimeout(function(){n.off(e.support.transition.end).detach()},500);n.one(e.support.transition.end,function(){clearTimeout(t),n.detach()})}var t=this,n=this.tip(),r=e.Event("hide");this.$element.trigger(r);if(r.isDefaultPrevented())return;return n.removeClass("in"),e.support.transition&&this.$tip.hasClass("fade")?i():n.detach(),this.$element.trigger("hidden"),this},fixTitle:function(){var e=this.$element;(e.attr("title")||typeof e.attr("data-original-title")!="string")&&e.attr("data-original-title",e.attr("title")||"").attr("title","")},hasContent:function(){return this.getTitle()},getPosition:function(){var t=this.$element[0];return e.extend({},typeof t.getBoundingClientRect=="function"?t.getBoundingClientRect():{width:t.offsetWidth,height:t.offsetHeight},this.$element.offset())},getTitle:function(){var e,t=this.$element,n=this.options;return e=t.attr("data-original-title")||(typeof n.title=="function"?n.title.call(t[0]):n.title),e},tip:function(){return this.$tip=this.$tip||e(this.options.template)},arrow:function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},validate:function(){this.$element[0].parentNode||(this.hide(),this.$element=null,this.options=null)},enable:function(){this.enabled=!0},disable:function(){this.enabled=!1},toggleEnabled:function(){this.enabled=!this.enabled},toggle:function(t){var n=t?e(t.currentTarget)[this.type](this._options).data(this.type):this;n.tip().hasClass("in")?n.hide():n.show()},destroy:function(){this.hide().$element.off("."+this.type).removeData(this.type)}};var n=e.fn.tooltip;e.fn.tooltip=function(n){return this.each(function(){var r=e(this),i=r.data("tooltip"),s=typeof n=="object"&&n;i||r.data("tooltip",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.tooltip.Constructor=t,e.fn.tooltip.defaults={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover focus",title:"",delay:0,html:!1,container:!1},e.fn.tooltip.noConflict=function(){return e.fn.tooltip=n,this}}(window.jQuery),!function(e){"use strict";var t=function(e,t){this.init("popover",e,t)};t.prototype=e.extend({},e.fn.tooltip.Constructor.prototype,{constructor:t,setContent:function(){var e=this.tip(),t=this.getTitle(),n=this.getContent();e.find(".popover-title")[this.options.html?"html":"text"](t),e.find(".popover-content")[this.options.html?"html":"text"](n),e.removeClass("fade top bottom left right in")},hasContent:function(){return this.getTitle()||this.getContent()},getContent:function(){var e,t=this.$element,n=this.options;return e=(typeof n.content=="function"?n.content.call(t[0]):n.content)||t.attr("data-content"),e},tip:function(){return this.$tip||(this.$tip=e(this.options.template)),this.$tip},destroy:function(){this.hide().$element.off("."+this.type).removeData(this.type)}});var n=e.fn.popover;e.fn.popover=function(n){return this.each(function(){var r=e(this),i=r.data("popover"),s=typeof n=="object"&&n;i||r.data("popover",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.popover.Constructor=t,e.fn.popover.defaults=e.extend({},e.fn.tooltip.defaults,{placement:"right",trigger:"click",content:"",template:'<div class="popover"><div class="arrow"></div><h3 class="popover-title"></h3><div class="popover-content"></div></div>'}),e.fn.popover.noConflict=function(){return e.fn.popover=n,this}}(window.jQuery),!function(e){"use strict";function t(t,n){var r=e.proxy(this.process,this),i=e(t).is("body")?e(window):e(t),s;this.options=e.extend({},e.fn.scrollspy.defaults,n),this.$scrollElement=i.on("scroll.scroll-spy.data-api",r),this.selector=(this.options.target||(s=e(t).attr("href"))&&s.replace(/.*(?=#[^\s]+$)/,"")||"")+" .nav li > a",this.$body=e("body"),this.refresh(),this.process()}t.prototype={constructor:t,refresh:function(){var t=this,n;this.offsets=e([]),this.targets=e([]),n=this.$body.find(this.selector).map(function(){var n=e(this),r=n.data("target")||n.attr("href"),i=/^#\w/.test(r)&&e(r);return i&&i.length&&[[i.position().top+(!e.isWindow(t.$scrollElement.get(0))&&t.$scrollElement.scrollTop()),r]]||null}).sort(function(e,t){return e[0]-t[0]}).each(function(){t.offsets.push(this[0]),t.targets.push(this[1])})},process:function(){var e=this.$scrollElement.scrollTop()+this.options.offset,t=this.$scrollElement[0].scrollHeight||this.$body[0].scrollHeight,n=t-this.$scrollElement.height(),r=this.offsets,i=this.targets,s=this.activeTarget,o;if(e>=n)return s!=(o=i.last()[0])&&this.activate(o);for(o=r.length;o--;)s!=i[o]&&e>=r[o]&&(!r[o+1]||e<=r[o+1])&&this.activate(i[o])},activate:function(t){var n,r;this.activeTarget=t,e(this.selector).parent(".active").removeClass("active"),r=this.selector+'[data-target="'+t+'"],'+this.selector+'[href="'+t+'"]',n=e(r).parent("li").addClass("active"),n.parent(".dropdown-menu").length&&(n=n.closest("li.dropdown").addClass("active")),n.trigger("activate")}};var n=e.fn.scrollspy;e.fn.scrollspy=function(n){return this.each(function(){var r=e(this),i=r.data("scrollspy"),s=typeof n=="object"&&n;i||r.data("scrollspy",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.scrollspy.Constructor=t,e.fn.scrollspy.defaults={offset:10},e.fn.scrollspy.noConflict=function(){return e.fn.scrollspy=n,this},e(window).on("load",function(){e('[data-spy="scroll"]').each(function(){var t=e(this);t.scrollspy(t.data())})})}(window.jQuery),!function(e){"use strict";var t=function(t){this.element=e(t)};t.prototype={constructor:t,show:function(){var t=this.element,n=t.closest("ul:not(.dropdown-menu)"),r=t.attr("data-target"),i,s,o;r||(r=t.attr("href"),r=r&&r.replace(/.*(?=#[^\s]*$)/,""));if(t.parent("li").hasClass("active"))return;i=n.find(".active:last a")[0],o=e.Event("show",{relatedTarget:i}),t.trigger(o);if(o.isDefaultPrevented())return;s=e(r),this.activate(t.parent("li"),n),this.activate(s,s.parent(),function(){t.trigger({type:"shown",relatedTarget:i})})},activate:function(t,n,r){function o(){i.removeClass("active").find("> .dropdown-menu > .active").removeClass("active"),t.addClass("active"),s?(t[0].offsetWidth,t.addClass("in")):t.removeClass("fade"),t.parent(".dropdown-menu")&&t.closest("li.dropdown").addClass("active"),r&&r()}var i=n.find("> .active"),s=r&&e.support.transition&&i.hasClass("fade");s?i.one(e.support.transition.end,o):o(),i.removeClass("in")}};var n=e.fn.tab;e.fn.tab=function(n){return this.each(function(){var r=e(this),i=r.data("tab");i||r.data("tab",i=new t(this)),typeof n=="string"&&i[n]()})},e.fn.tab.Constructor=t,e.fn.tab.noConflict=function(){return e.fn.tab=n,this},e(document).on("click.tab.data-api",'[data-toggle="tab"], [data-toggle="pill"]',function(t){t.preventDefault(),e(this).tab("show")})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.options=e.extend({},e.fn.typeahead.defaults,n),this.matcher=this.options.matcher||this.matcher,this.sorter=this.options.sorter||this.sorter,this.highlighter=this.options.highlighter||this.highlighter,this.updater=this.options.updater||this.updater,this.source=this.options.source,this.$menu=e(this.options.menu),this.shown=!1,this.listen()};t.prototype={constructor:t,select:function(){var e=this.$menu.find(".active").attr("data-value");return this.$element.val(this.updater(e)).change(),this.hide()},updater:function(e){return e},show:function(){var t=e.extend({},this.$element.position(),{height:this.$element[0].offsetHeight});return this.$menu.insertAfter(this.$element).css({top:t.top+t.height,left:t.left}).show(),this.shown=!0,this},hide:function(){return this.$menu.hide(),this.shown=!1,this},lookup:function(t){var n;return this.query=this.$element.val(),!this.query||this.query.length<this.options.minLength?this.shown?this.hide():this:(n=e.isFunction(this.source)?this.source(this.query,e.proxy(this.process,this)):this.source,n?this.process(n):this)},process:function(t){var n=this;return t=e.grep(t,function(e){return n.matcher(e)}),t=this.sorter(t),t.length?this.render(t.slice(0,this.options.items)).show():this.shown?this.hide():this},matcher:function(e){return~e.toLowerCase().indexOf(this.query.toLowerCase())},sorter:function(e){var t=[],n=[],r=[],i;while(i=e.shift())i.toLowerCase().indexOf(this.query.toLowerCase())?~i.indexOf(this.query)?n.push(i):r.push(i):t.push(i);return t.concat(n,r)},highlighter:function(e){var t=this.query.replace(/[\-\[\]{}()*+?.,\\\^$|#\s]/g,"\\$&");return e.replace(new RegExp("("+t+")","ig"),function(e,t){return"<strong>"+t+"</strong>"})},render:function(t){var n=this;return t=e(t).map(function(t,r){return t=e(n.options.item).attr("data-value",r),t.find("a").html(n.highlighter(r)),t[0]}),t.first().addClass("active"),this.$menu.html(t),this},next:function(t){var n=this.$menu.find(".active").removeClass("active"),r=n.next();r.length||(r=e(this.$menu.find("li")[0])),r.addClass("active")},prev:function(e){var t=this.$menu.find(".active").removeClass("active"),n=t.prev();n.length||(n=this.$menu.find("li").last()),n.addClass("active")},listen:function(){this.$element.on("focus",e.proxy(this.focus,this)).on("blur",e.proxy(this.blur,this)).on("keypress",e.proxy(this.keypress,this)).on("keyup",e.proxy(this.keyup,this)),this.eventSupported("keydown")&&this.$element.on("keydown",e.proxy(this.keydown,this)),this.$menu.on("click",e.proxy(this.click,this)).on("mouseenter","li",e.proxy(this.mouseenter,this)).on("mouseleave","li",e.proxy(this.mouseleave,this))},eventSupported:function(e){var t=e in this.$element;return t||(this.$element.setAttribute(e,"return;"),t=typeof this.$element[e]=="function"),t},move:function(e){if(!this.shown)return;switch(e.keyCode){case 9:case 13:case 27:e.preventDefault();break;case 38:e.preventDefault(),this.prev();break;case 40:e.preventDefault(),this.next()}e.stopPropagation()},keydown:function(t){this.suppressKeyPressRepeat=~e.inArray(t.keyCode,[40,38,9,13,27]),this.move(t)},keypress:function(e){if(this.suppressKeyPressRepeat)return;this.move(e)},keyup:function(e){switch(e.keyCode){case 40:case 38:case 16:case 17:case 18:break;case 9:case 13:if(!this.shown)return;this.select();break;case 27:if(!this.shown)return;this.hide();break;default:this.lookup()}e.stopPropagation(),e.preventDefault()},focus:function(e){this.focused=!0},blur:function(e){this.focused=!1,!this.mousedover&&this.shown&&this.hide()},click:function(e){e.stopPropagation(),e.preventDefault(),this.select(),this.$element.focus()},mouseenter:function(t){this.mousedover=!0,this.$menu.find(".active").removeClass("active"),e(t.currentTarget).addClass("active")},mouseleave:function(e){this.mousedover=!1,!this.focused&&this.shown&&this.hide()}};var n=e.fn.typeahead;e.fn.typeahead=function(n){return this.each(function(){var r=e(this),i=r.data("typeahead"),s=typeof n=="object"&&n;i||r.data("typeahead",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.typeahead.defaults={source:[],items:8,menu:'<ul class="typeahead dropdown-menu"></ul>',item:'<li><a href="#"></a></li>',minLength:1},e.fn.typeahead.Constructor=t,e.fn.typeahead.noConflict=function(){return e.fn.typeahead=n,this},e(document).on("focus.typeahead.data-api",'[data-provide="typeahead"]',function(t){var n=e(this);if(n.data("typeahead"))return;n.typeahead(n.data())})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.options=e.extend({},e.fn.affix.defaults,n),this.$window=e(window).on("scroll.affix.data-api",e.proxy(this.checkPosition,this)).on("click.affix.data-api",e.proxy(function(){setTimeout(e.proxy(this.checkPosition,this),1)},this)),this.$element=e(t),this.checkPosition()};t.prototype.checkPosition=function(){if(!this.$element.is(":visible"))return;var t=e(document).height(),n=this.$window.scrollTop(),r=this.$element.offset(),i=this.options.offset,s=i.bottom,o=i.top,u="affix affix-top affix-bottom",a;typeof i!="object"&&(s=o=i),typeof o=="function"&&(o=i.top()),typeof s=="function"&&(s=i.bottom()),a=this.unpin!=null&&n+this.unpin<=r.top?!1:s!=null&&r.top+this.$element.height()>=t-s?"bottom":o!=null&&n<=o?"top":!1;if(this.affixed===a)return;this.affixed=a,this.unpin=a=="bottom"?r.top-n:null,this.$element.removeClass(u).addClass("affix"+(a?"-"+a:""))};var n=e.fn.affix;e.fn.affix=function(n){return this.each(function(){var r=e(this),i=r.data("affix"),s=typeof n=="object"&&n;i||r.data("affix",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.affix.Constructor=t,e.fn.affix.defaults={offset:0},e.fn.affix.noConflict=function(){return e.fn.affix=n,this},e(window).on("load",function(){e('[data-spy="affix"]').each(function(){var t=e(this),n=t.data();n.offset=n.offset||{},n.offsetBottom&&(n.offset.bottom=n.offsetBottom),n.offsetTop&&(n.offset.top=n.offsetTop),t.affix(n)})})}(window.jQuery);
\ No newline at end of file
+!function(a){a(function(){a.support.transition=function(){var a=function(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"},c;for(c in b)if(a.style[c]!==undefined)return b[c]}();return a&&{end:a}}()})}(window.jQuery),!function(a){var b=function(b,c){this.options=c,this.$element=a(b).delegate('[data-dismiss="modal"]',"click.dismiss.modal",a.proxy(this.hide,this)),this.options.remote&&this.$element.find(".modal-body").load(this.options.remote)};b.prototype={constructor:b,toggle:function(){return this[this.isShown?"hide":"show"]()},show:function(){var b=this,c=a.Event("show");this.$element.trigger(c);if(this.isShown||c.isDefaultPrevented())return;this.isShown=!0,this.escape(),this.backdrop(function(){var c=a.support.transition&&b.$element.hasClass("fade");b.$element.parent().length||b.$element.appendTo(document.body),b.$element.show(),c&&b.$element[0].offsetWidth,b.$element.addClass("in").attr("aria-hidden",!1),b.enforceFocus(),c?b.$element.one(a.support.transition.end,function(){b.$element.focus().trigger("shown")}):b.$element.focus().trigger("shown")})},hide:function(b){b&&b.preventDefault();var c=this;b=a.Event("hide"),this.$element.trigger(b);if(!this.isShown||b.isDefaultPrevented())return;this.isShown=!1,this.escape(),a(document).off("focusin.modal"),this.$element.removeClass("in").attr("aria-hidden",!0),a.support.transition&&this.$element.hasClass("fade")?this.hideWithTransition():this.hideModal()},enforceFocus:function(){var b=this;a(document).on("focusin.modal",function(a){b.$element[0]!==a.target&&!b.$element.has(a.target).length&&b.$element.focus()})},escape:function(){var a=this;this.isShown&&this.options.keyboard?this.$element.on("keyup.dismiss.modal",function(b){b.which==27&&a.hide()}):this.isShown||this.$element.off("keyup.dismiss.modal")},hideWithTransition:function(){var b=this,c=setTimeout(function(){b.$element.off(a.support.transition.end),b.hideModal()},500);this.$element.one(a.support.transition.end,function(){clearTimeout(c),b.hideModal()})},hideModal:function(){var a=this;this.$element.hide(),this.backdrop(function(){a.removeBackdrop(),a.$element.trigger("hidden")})},removeBackdrop:function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},backdrop:function(b){var c=this,d=this.$element.hasClass("fade")?"fade":"";if(this.isShown&&this.options.backdrop){var e=a.support.transition&&d;this.$backdrop=a('<div class="modal-backdrop '+d+'" />').appendTo(document.body),this.$backdrop.click(this.options.backdrop=="static"?a.proxy(this.$element[0].focus,this.$element[0]):a.proxy(this.hide,this)),e&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in");if(!b)return;e?this.$backdrop.one(a.support.transition.end,b):b()}else!this.isShown&&this.$backdrop?(this.$backdrop.removeClass("in"),a.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one(a.support.transition.end,b):b()):b&&b()}};var c=a.fn.modal;a.fn.modal=function(c){return this.each(function(){var d=a(this),e=d.data("modal"),f=a.extend({},a.fn.modal.defaults,d.data(),typeof c=="object"&&c);e||d.data("modal",e=new b(this,f)),typeof c=="string"?e[c]():f.show&&e.show()})},a.fn.modal.defaults={backdrop:!0,keyboard:!0,show:!0},a.fn.modal.Constructor=b,a.fn.modal.noConflict=function(){return a.fn.modal=c,this},a(document).on("click.modal.data-api",'[data-toggle="modal"]',function(b){var c=a(this),d=c.attr("href"),e=a(c.attr("data-target")||d&&d.replace(/.*(?=#[^\s]+$)/,"")),f=e.data("modal")?"toggle":a.extend({remote:!/#/.test(d)&&d},e.data(),c.data());b.preventDefault(),e.modal(f).one("hide",function(){c.focus()})})}(window.jQuery),!function(a){function d(){a(".dropdown-backdrop").remove(),a(b).each(function(){e(a(this)).removeClass("open")})}function e(b){var c=b.attr("data-target"),d;c||(c=b.attr("href"),c=c&&/#/.test(c)&&c.replace(/.*(?=#[^\s]*$)/,"")),d=c&&a(c);if(!d||!d.length)d=b.parent();return d}var b="[data-toggle=dropdown]",c=function(b){var c=a(b).on("click.dropdown.data-api",this.toggle);a("html").on("click.dropdown.data-api",function(){c.parent().removeClass("open")})};c.prototype={constructor:c,toggle:function(b){var c=a(this),f,g;if(c.is(".disabled, :disabled"))return;return f=e(c),g=f.hasClass("open"),d(),g||("ontouchstart"in document.documentElement&&a('<div class="dropdown-backdrop"/>').insertBefore(a(this)).on("click",d),f.toggleClass("open")),c.focus(),!1},keydown:function(c){var d,f,g,h,i,j;if(!/(38|40|27)/.test(c.keyCode))return;d=a(this),c.preventDefault(),c.stopPropagation();if(d.is(".disabled, :disabled"))return;h=e(d),i=h.hasClass("open");if(!i||i&&c.keyCode==27)return c.which==27&&h.find(b).focus(),d.click();f=a("[role=menu] li:not(.divider):visible a",h);if(!f.length)return;j=f.index(f.filter(":focus")),c.keyCode==38&&j>0&&j--,c.keyCode==40&&j<f.length-1&&j++,~j||(j=0),f.eq(j).focus()}};var f=a.fn.dropdown;a.fn.dropdown=function(b){return this.each(function(){var d=a(this),e=d.data("dropdown");e||d.data("dropdown",e=new c(this)),typeof b=="string"&&e[b].call(d)})},a.fn.dropdown.Constructor=c,a.fn.dropdown.noConflict=function(){return a.fn.dropdown=f,this},a(document).on("click.dropdown.data-api",d).on("click.dropdown.data-api",".dropdown form",function(a){a.stopPropagation()}).on("click.dropdown.data-api",b,c.prototype.toggle).on("keydown.dropdown.data-api",b+", [role=menu]",c.prototype.keydown)}(window.jQuery),!function(a){function b(b,c){var d=a.proxy(this.process,this),e=a(b).is("body")?a(window):a(b),f;this.options=a.extend({},a.fn.scrollspy.defaults,c),this.$scrollElement=e.on("scroll.scroll-spy.data-api",d),this.selector=(this.options.target||(f=a(b).attr("href"))&&f.replace(/.*(?=#[^\s]+$)/,"")||"")+" .nav li > a",this.$body=a("body"),this.refresh(),this.process()}b.prototype={constructor:b,refresh:function(){var b=this,c;this.offsets=a([]),this.targets=a([]),c=this.$body.find(this.selector).map(function(){var c=a(this),d=c.data("target")||c.attr("href"),e=/^#\w/.test(d)&&a(d);return e&&e.length&&[[e.position().top+(!a.isWindow(b.$scrollElement.get(0))&&b.$scrollElement.scrollTop()),d]]||null}).sort(function(a,b){return a[0]-b[0]}).each(function(){b.offsets.push(this[0]),b.targets.push(this[1])})},process:function(){var a=this.$scrollElement.scrollTop()+this.options.offset,b=this.$scrollElement[0].scrollHeight||this.$body[0].scrollHeight,c=b-this.$scrollElement.height(),d=this.offsets,e=this.targets,f=this.activeTarget,g;if(a>=c)return f!=(g=e.last()[0])&&this.activate(g);for(g=d.length;g--;)f!=e[g]&&a>=d[g]&&(!d[g+1]||a<=d[g+1])&&this.activate(e[g])},activate:function(b){var c,d;this.activeTarget=b,a(this.selector).parent(".active").removeClass("active"),d=this.selector+'[data-target="'+b+'"],'+this.selector+'[href="'+b+'"]',c=a(d).parent("li").addClass("active"),c.parent(".dropdown-menu").length&&(c=c.closest("li.dropdown").addClass("active")),c.trigger("activate")}};var c=a.fn.scrollspy;a.fn.scrollspy=function(c){return this.each(function(){var d=a(this),e=d.data("scrollspy"),f=typeof c=="object"&&c;e||d.data("scrollspy",e=new b(this,f)),typeof c=="string"&&e[c]()})},a.fn.scrollspy.Constructor=b,a.fn.scrollspy.defaults={offset:10},a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=c,this},a(window).on("load",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);b.scrollspy(b.data())})})}(window.jQuery),!function(a){var b=function(b){this.element=a(b)};b.prototype={constructor:b,show:function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.attr("data-target"),e,f,g;d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,""));if(b.parent("li").hasClass("active"))return;e=c.find(".active:last a")[0],g=a.Event("show",{relatedTarget:e}),b.trigger(g);if(g.isDefaultPrevented())return;f=a(d),this.activate(b.parent("li"),c),this.activate(f,f.parent(),function(){b.trigger({type:"shown",relatedTarget:e})})},activate:function(b,c,d){function g(){e.removeClass("active").find("> .dropdown-menu > .active").removeClass("active"),b.addClass("active"),f?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu")&&b.closest("li.dropdown").addClass("active"),d&&d()}var e=c.find("> .active"),f=d&&a.support.transition&&e.hasClass("fade");f?e.one(a.support.transition.end,g):g(),e.removeClass("in")}};var c=a.fn.tab;a.fn.tab=function(c){return this.each(function(){var d=a(this),e=d.data("tab");e||d.data("tab",e=new b(this)),typeof c=="string"&&e[c]()})},a.fn.tab.Constructor=b,a.fn.tab.noConflict=function(){return a.fn.tab=c,this},a(document).on("click.tab.data-api",'[data-toggle="tab"], [data-toggle="pill"]',function(b){b.preventDefault(),a(this).tab("show")})}(window.jQuery),!function(a){var b=function(a,b){this.init("tooltip",a,b)};b.prototype={constructor:b,init:function(b,c,d){var e,f,g,h,i;this.type=b,this.$element=a(c),this.options=this.getOptions(d),this.enabled=!0,g=this.options.trigger.split(" ");for(i=g.length;i--;)h=g[i],h=="click"?this.$element.on("click."+this.type,this.options.selector,a.proxy(this.toggle,this)):h!="manual"&&(e=h=="hover"?"mouseenter":"focus",f=h=="hover"?"mouseleave":"blur",this.$element.on(e+"."+this.type,this.options.selector,a.proxy(this.enter,this)),this.$element.on(f+"."+this.type,this.options.selector,a.proxy(this.leave,this)));this.options.selector?this._options=a.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},getOptions:function(b){return b=a.extend({},a.fn[this.type].defaults,this.$element.data(),b),b.delay&&typeof b.delay=="number"&&(b.delay={show:b.delay,hide:b.delay}),b},enter:function(b){var c=a.fn[this.type].defaults,d={},e;this._options&&a.each(this._options,function(a,b){c[a]!=b&&(d[a]=b)},this),e=a(b.currentTarget)[this.type](d).data(this.type);if(!e.options.delay||!e.options.delay.show)return e.show();clearTimeout(this.timeout),e.hoverState="in",this.timeout=setTimeout(function(){e.hoverState=="in"&&e.show()},e.options.delay.show)},leave:function(b){var c=a(b.currentTarget)[this.type](this._options).data(this.type);this.timeout&&clearTimeout(this.timeout);if(!c.options.delay||!c.options.delay.hide)return c.hide();c.hoverState="out",this.timeout=setTimeout(function(){c.hoverState=="out"&&c.hide()},c.options.delay.hide)},show:function(){var b,c,d,e,f,g,h=a.Event("show");if(this.hasContent()&&this.enabled){this.$element.trigger(h);if(h.isDefaultPrevented())return;b=this.tip(),this.setContent(),this.options.animation&&b.addClass("fade"),f=typeof this.options.placement=="function"?this.options.placement.call(this,b[0],this.$element[0]):this.options.placement,b.detach().css({top:0,left:0,display:"block"}),this.options.container?b.appendTo(this.options.container):b.insertAfter(this.$element),c=this.getPosition(),d=b[0].offsetWidth,e=b[0].offsetHeight;switch(f){case"bottom":g={top:c.top+c.height,left:c.left+c.width/2-d/2};break;case"top":g={top:c.top-e,left:c.left+c.width/2-d/2};break;case"left":g={top:c.top+c.height/2-e/2,left:c.left-d};break;case"right":g={top:c.top+c.height/2-e/2,left:c.left+c.width}}this.applyPlacement(g,f),this.$element.trigger("shown")}},applyPlacement:function(a,b){var c=this.tip(),d=c[0].offsetWidth,e=c[0].offsetHeight,f,g,h,i;c.offset(a).addClass(b).addClass("in"),f=c[0].offsetWidth,g=c[0].offsetHeight,b=="top"&&g!=e&&(a.top=a.top+e-g,i=!0),b=="bottom"||b=="top"?(h=0,a.left<0&&(h=a.left*-2,a.left=0,c.offset(a),f=c[0].offsetWidth,g=c[0].offsetHeight),this.replaceArrow(h-d+f,f,"left")):this.replaceArrow(g-e,g,"top"),i&&c.offset(a)},replaceArrow:function(a,b,c){this.arrow().css(c,a?50*(1-a/b)+"%":"")},setContent:function(){var a=this.tip(),b=this.getTitle();a.find(".tooltip-inner")[this.options.html?"html":"text"](b),a.removeClass("fade in top bottom left right")},hide:function(){function e(){var b=setTimeout(function(){c.off(a.support.transition.end).detach()},500);c.one(a.support.transition.end,function(){clearTimeout(b),c.detach()})}var b=this,c=this.tip(),d=a.Event("hide");this.$element.trigger(d);if(d.isDefaultPrevented())return;return c.removeClass("in"),a.support.transition&&this.$tip.hasClass("fade")?e():c.detach(),this.$element.trigger("hidden"),this},fixTitle:function(){var a=this.$element;(a.attr("title")||typeof a.attr("data-original-title")!="string")&&a.attr("data-original-title",a.attr("title")||"").attr("title","")},hasContent:function(){return this.getTitle()},getPosition:function(){var b=this.$element[0];return a.extend({},typeof b.getBoundingClientRect=="function"?b.getBoundingClientRect():{width:b.offsetWidth,height:b.offsetHeight},this.$element.offset())},getTitle:function(){var a,b=this.$element,c=this.options;return a=b.attr("data-original-title")||(typeof c.title=="function"?c.title.call(b[0]):c.title),a},tip:function(){return this.$tip=this.$tip||a(this.options.template)},arrow:function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},validate:function(){this.$element[0].parentNode||(this.hide(),this.$element=null,this.options=null)},enable:function(){this.enabled=!0},disable:function(){this.enabled=!1},toggleEnabled:function(){this.enabled=!this.enabled},toggle:function(b){var c=b?a(b.currentTarget)[this.type](this._options).data(this.type):this;c.tip().hasClass("in")?c.hide():c.show()},destroy:function(){this.hide().$element.off("."+this.type).removeData(this.type)}};var c=a.fn.tooltip;a.fn.tooltip=function(c){return this.each(function(){var d=a(this),e=d.data("tooltip"),f=typeof c=="object"&&c;e||d.data("tooltip",e=new b(this,f)),typeof c=="string"&&e[c]()})},a.fn.tooltip.Constructor=b,a.fn.tooltip.defaults={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover focus",title:"",delay:0,html:!1,container:!1},a.fn.tooltip.noConflict=function(){return a.fn.tooltip=c,this}}(window.jQuery),!function(a){var b=function(a,b){this.init("popover",a,b)};b.prototype=a.extend({},a.fn.tooltip.Constructor.prototype,{constructor:b,setContent:function(){var a=this.tip(),b=this.getTitle(),c=this.getContent();a.find(".popover-title")[this.options.html?"html":"text"](b),a.find(".popover-content")[this.options.html?"html":"text"](c),a.removeClass("fade top bottom left right in")},hasContent:function(){return this.getTitle()||this.getContent()},getContent:function(){var a,b=this.$element,c=this.options;return a=(typeof c.content=="function"?c.content.call(b[0]):c.content)||b.attr("data-content"),a},tip:function(){return this.$tip||(this.$tip=a(this.options.template)),this.$tip},destroy:function(){this.hide().$element.off("."+this.type).removeData(this.type)}});var c=a.fn.popover;a.fn.popover=function(c){return this.each(function(){var d=a(this),e=d.data("popover"),f=typeof c=="object"&&c;e||d.data("popover",e=new b(this,f)),typeof c=="string"&&e[c]()})},a.fn.popover.Constructor=b,a.fn.popover.defaults=a.extend({},a.fn.tooltip.defaults,{placement:"right",trigger:"click",content:"",template:'<div class="popover"><div class="arrow"></div><h3 class="popover-title"></h3><div class="popover-content"></div></div>'}),a.fn.popover.noConflict=function(){return a.fn.popover=c,this}}(window.jQuery),!function(a){var b=function(b,c){this.options=a.extend({},a.fn.affix.defaults,c),this.$window=a(window).on("scroll.affix.data-api",a.proxy(this.checkPosition,this)).on("click.affix.data-api",a.proxy(function(){setTimeout(a.proxy(this.checkPosition,this),1)},this)),this.$element=a(b),this.checkPosition()};b.prototype.checkPosition=function(){if(!this.$element.is(":visible"))return;var b=a(document).height(),c=this.$window.scrollTop(),d=this.$element.offset(),e=this.options.offset,f=e.bottom,g=e.top,h="affix affix-top affix-bottom",i;typeof e!="object"&&(f=g=e),typeof g=="function"&&(g=e.top()),typeof f=="function"&&(f=e.bottom()),i=this.unpin!=null&&c+this.unpin<=d.top?!1:f!=null&&d.top+this.$element.height()>=b-f?"bottom":g!=null&&c<=g?"top":!1;if(this.affixed===i)return;this.affixed=i,this.unpin=i=="bottom"?d.top-c:null,this.$element.removeClass(h).addClass("affix"+(i?"-"+i:""))};var c=a.fn.affix;a.fn.affix=function(c){return this.each(function(){var d=a(this),e=d.data("affix"),f=typeof c=="object"&&c;e||d.data("affix",e=new b(this,f)),typeof c=="string"&&e[c]()})},a.fn.affix.Constructor=b,a.fn.affix.defaults={offset:0},a.fn.affix.noConflict=function(){return a.fn.affix=c,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var b=a(this),c=b.data();c.offset=c.offset||{},c.offsetBottom&&(c.offset.bottom=c.offsetBottom),c.offsetTop&&(c.offset.top=c.offsetTop),b.affix(c)})})}(window.jQuery),!function(a){var b='[data-dismiss="alert"]',c=function(c){a(c).on("click",b,this.close)};c.prototype.close=function(b){function f(){e.trigger("closed").remove()}var c=a(this),d=c.attr("data-target"),e;d||(d=c.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),e=a(d),b&&b.preventDefault(),e.length||(e=c.hasClass("alert")?c:c.parent()),e.trigger(b=a.Event("close"));if(b.isDefaultPrevented())return;e.removeClass("in"),a.support.transition&&e.hasClass("fade")?e.on(a.support.transition.end,f):f()};var d=a.fn.alert;a.fn.alert=function(b){return this.each(function(){var d=a(this),e=d.data("alert");e||d.data("alert",e=new c(this)),typeof b=="string"&&e[b].call(d)})},a.fn.alert.Constructor=c,a.fn.alert.noConflict=function(){return a.fn.alert=d,this},a(document).on("click.alert.data-api",b,c.prototype.close)}(window.jQuery),!function(a){var b=function(b,c){this.$element=a(b),this.options=a.extend({},a.fn.button.defaults,c)};b.prototype.setState=function(a){var b="disabled",c=this.$element,d=c.data(),e=c.is("input")?"val":"html";a+="Text",d.resetText||c.data("resetText",c[e]()),c[e](d[a]||this.options[a]),setTimeout(function(){a=="loadingText"?c.addClass(b).attr(b,b):c.removeClass(b).removeAttr(b)},0)},b.prototype.toggle=function(){var a=this.$element.closest('[data-toggle="buttons-radio"]');a&&a.find(".active").removeClass("active"),this.$element.toggleClass("active")};var c=a.fn.button;a.fn.button=function(c){return this.each(function(){var d=a(this),e=d.data("button"),f=typeof c=="object"&&c;e||d.data("button",e=new b(this,f)),c=="toggle"?e.toggle():c&&e.setState(c)})},a.fn.button.defaults={loadingText:"loading..."},a.fn.button.Constructor=b,a.fn.button.noConflict=function(){return a.fn.button=c,this},a(document).on("click.button.data-api","[data-toggle^=button]",function(b){var c=a(b.target);c.hasClass("btn")||(c=c.closest(".btn")),c.button("toggle")})}(window.jQuery),!function(a){var b=function(b,c){this.$element=a(b),this.options=a.extend({},a.fn.collapse.defaults,c),this.options.parent&&(this.$parent=a(this.options.parent)),this.options.toggle&&this.toggle()};b.prototype={constructor:b,dimension:function(){var a=this.$element.hasClass("width");return a?"width":"height"},show:function(){var b,c,d,e;if(this.transitioning||this.$element.hasClass("in"))return;b=this.dimension(),c=a.camelCase(["scroll",b].join("-")),d=this.$parent&&this.$parent.find("> .accordion-group > .in");if(d&&d.length){e=d.data("collapse");if(e&&e.transitioning)return;d.collapse("hide"),e||d.data("collapse",null)}this.$element[b](0),this.transition("addClass",a.Event("show"),"shown"),a.support.transition&&this.$element[b](this.$element[0][c])},hide:function(){var b;if(this.transitioning||!this.$element.hasClass("in"))return;b=this.dimension(),this.reset(this.$element[b]()),this.transition("removeClass",a.Event("hide"),"hidden"),this.$element[b](0)},reset:function(a){var b=this.dimension();return this.$element.removeClass("collapse")[b](a||"auto")[0].offsetWidth,this.$element[a!==null?"addClass":"removeClass"]("collapse"),this},transition:function(b,c,d){var e=this,f=function(){c.type=="show"&&e.reset(),e.transitioning=0,e.$element.trigger(d)};this.$element.trigger(c);if(c.isDefaultPrevented())return;this.transitioning=1,this.$element[b]("in"),a.support.transition&&this.$element.hasClass("collapse")?this.$element.one(a.support.transition.end,f):f()},toggle:function(){this[this.$element.hasClass("in")?"hide":"show"]()}};var c=a.fn.collapse;a.fn.collapse=function(c){return this.each(function(){var d=a(this),e=d.data("collapse"),f=a.extend({},a.fn.collapse.defaults,d.data(),typeof c=="object"&&c);e||d.data("collapse",e=new b(this,f)),typeof c=="string"&&e[c]()})},a.fn.collapse.defaults={toggle:!0},a.fn.collapse.Constructor=b,a.fn.collapse.noConflict=function(){return a.fn.collapse=c,this},a(document).on("click.collapse.data-api","[data-toggle=collapse]",function(b){var c=a(this),d,e=c.attr("data-target")||b.preventDefault()||(d=c.attr("href"))&&d.replace(/.*(?=#[^\s]+$)/,""),f=a(e).data("collapse")?"toggle":c.data();c[a(e).hasClass("in")?"addClass":"removeClass"]("collapsed"),a(e).collapse(f)})}(window.jQuery),!function(a){var b=function(b,c){this.$element=a(b),this.$indicators=this.$element.find(".carousel-indicators"),this.options=c,this.options.pause=="hover"&&this.$element.on("mouseenter",a.proxy(this.pause,this)).on("mouseleave",a.proxy(this.cycle,this))};b.prototype={cycle:function(b){return b||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(a.proxy(this.next,this),this.options.interval)),this},getActiveIndex:function(){return this.$active=this.$element.find(".item.active"),this.$items=this.$active.parent().children(),this.$items.index(this.$active)},to:function(b){var c=this.getActiveIndex(),d=this;if(b>this.$items.length-1||b<0)return;return this.sliding?this.$element.one("slid",function(){d.to(b)}):c==b?this.pause().cycle():this.slide(b>c?"next":"prev",a(this.$items[b]))},pause:function(b){return b||(this.paused=!0),this.$element.find(".next, .prev").length&&a.support.transition.end&&(this.$element.trigger(a.support.transition.end),this.cycle(!0)),clearInterval(this.interval),this.interval=null,this},next:function(){if(this.sliding)return;return this.slide("next")},prev:function(){if(this.sliding)return;return this.slide("prev")},slide:function(b,c){var d=this.$element.find(".item.active"),e=c||d[b](),f=this.interval,g=b=="next"?"left":"right",h=b=="next"?"first":"last",i=this,j;this.sliding=!0,f&&this.pause(),e=e.length?e:this.$element.find(".item")[h](),j=a.Event("slide",{relatedTarget:e[0],direction:g});if(e.hasClass("active"))return;this.$indicators.length&&(this.$indicators.find(".active").removeClass("active"),this.$element.one("slid",function(){var b=a(i.$indicators.children()[i.getActiveIndex()]);b&&b.addClass("active")}));if(a.support.transition&&this.$element.hasClass("slide")){this.$element.trigger(j);if(j.isDefaultPrevented())return;e.addClass(b),e[0].offsetWidth,d.addClass(g),e.addClass(g),this.$element.one(a.support.transition.end,function(){e.removeClass([b,g].join(" ")).addClass("active"),d.removeClass(["active",g].join(" ")),i.sliding=!1,setTimeout(function(){i.$element.trigger("slid")},0)})}else{this.$element.trigger(j);if(j.isDefaultPrevented())return;d.removeClass("active"),e.addClass("active"),this.sliding=!1,this.$element.trigger("slid")}return f&&this.cycle(),this}};var c=a.fn.carousel;a.fn.carousel=function(c){return this.each(function(){var d=a(this),e=d.data("carousel"),f=a.extend({},a.fn.carousel.defaults,typeof c=="object"&&c),g=typeof c=="string"?c:f.slide;e||d.data("carousel",e=new b(this,f)),typeof c=="number"?e.to(c):g?e[g]():f.interval&&e.pause().cycle()})},a.fn.carousel.defaults={interval:5e3,pause:"hover"},a.fn.carousel.Constructor=b,a.fn.carousel.noConflict=function(){return a.fn.carousel=c,this},a(document).on("click.carousel.data-api","[data-slide], [data-slide-to]",function(b){var c=a(this),d,e=a(c.attr("data-target")||(d=c.attr("href"))&&d.replace(/.*(?=#[^\s]+$)/,"")),f=a.extend({},e.data(),c.data()),g;e.carousel(f),(g=c.attr("data-slide-to"))&&e.data("carousel").pause().to(g).cycle(),b.preventDefault()})}(window.jQuery),!function(a){var b=function(b,c){this.$element=a(b),this.options=a.extend({},a.fn.typeahead.defaults,c),this.matcher=this.options.matcher||this.matcher,this.sorter=this.options.sorter||this.sorter,this.highlighter=this.options.highlighter||this.highlighter,this.updater=this.options.updater||this.updater,this.source=this.options.source,this.$menu=a(this.options.menu),this.shown=!1,this.listen()};b.prototype={constructor:b,select:function(){var a=this.$menu.find(".active").attr("data-value");return this.$element.val(this.updater(a)).change(),this.hide()},updater:function(a){return a},show:function(){var b=a.extend({},this.$element.position(),{height:this.$element[0].offsetHeight});return this.$menu.insertAfter(this.$element).css({top:b.top+b.height,left:b.left}).show(),this.shown=!0,this},hide:function(){return this.$menu.hide(),this.shown=!1,this},lookup:function(b){var c;return this.query=this.$element.val(),!this.query||this.query.length<this.options.minLength?this.shown?this.hide():this:(c=a.isFunction(this.source)?this.source(this.query,a.proxy(this.process,this)):this.source,c?this.process(c):this)},process:function(b){var c=this;return b=a.grep(b,function(a){return c.matcher(a)}),b=this.sorter(b),b.length?this.render(b.slice(0,this.options.items)).show():this.shown?this.hide():this},matcher:function(a){return~a.toLowerCase().indexOf(this.query.toLowerCase())},sorter:function(a){var b=[],c=[],d=[],e;while(e=a.shift())e.toLowerCase().indexOf(this.query.toLowerCase())?~e.indexOf(this.query)?c.push(e):d.push(e):b.push(e);return b.concat(c,d)},highlighter:function(a){var b=this.query.replace(/[\-\[\]{}()*+?.,\\\^$|#\s]/g,"\\$&");return a.replace(new RegExp("("+b+")","ig"),function(a,b){return"<strong>"+b+"</strong>"})},render:function(b){var c=this;return b=a(b).map(function(b,d){return b=a(c.options.item).attr("data-value",d),b.find("a").html(c.highlighter(d)),b[0]}),b.first().addClass("active"),this.$menu.html(b),this},next:function(b){var c=this.$menu.find(".active").removeClass("active"),d=c.next();d.length||(d=a(this.$menu.find("li")[0])),d.addClass("active")},prev:function(a){var b=this.$menu.find(".active").removeClass("active"),c=b.prev();c.length||(c=this.$menu.find("li").last()),c.addClass("active")},listen:function(){this.$element.on("focus",a.proxy(this.focus,this)).on("blur",a.proxy(this.blur,this)).on("keypress",a.proxy(this.keypress,this)).on("keyup",a.proxy(this.keyup,this)),this.eventSupported("keydown")&&this.$element.on("keydown",a.proxy(this.keydown,this)),this.$menu.on("click",a.proxy(this.click,this)).on("mouseenter","li",a.proxy(this.mouseenter,this)).on("mouseleave","li",a.proxy(this.mouseleave,this))},eventSupported:function(a){var b=a in this.$element;return b||(this.$element.setAttribute(a,"return;"),b=typeof this.$element[a]=="function"),b},move:function(a){if(!this.shown)return;switch(a.keyCode){case 9:case 13:case 27:a.preventDefault();break;case 38:a.preventDefault(),this.prev();break;case 40:a.preventDefault(),this.next()}a.stopPropagation()},keydown:function(b){this.suppressKeyPressRepeat=~a.inArray(b.keyCode,[40,38,9,13,27]),this.move(b)},keypress:function(a){if(this.suppressKeyPressRepeat)return;this.move(a)},keyup:function(a){switch(a.keyCode){case 40:case 38:case 16:case 17:case 18:break;case 9:case 13:if(!this.shown)return;this.select();break;case 27:if(!this.shown)return;this.hide();break;default:this.lookup()}a.stopPropagation(),a.preventDefault()},focus:function(a){this.focused=!0},blur:function(a){this.focused=!1,!this.mousedover&&this.shown&&this.hide()},click:function(a){a.stopPropagation(),a.preventDefault(),this.select(),this.$element.focus()},mouseenter:function(b){this.mousedover=!0,this.$menu.find(".active").removeClass("active"),a(b.currentTarget).addClass("active")},mouseleave:function(a){this.mousedover=!1,!this.focused&&this.shown&&this.hide()}};var c=a.fn.typeahead;a.fn.typeahead=function(c){return this.each(function(){var d=a(this),e=d.data("typeahead"),f=typeof c=="object"&&c;e||d.data("typeahead",e=new b(this,f)),typeof c=="string"&&e[c]()})},a.fn.typeahead.defaults={source:[],items:8,menu:'<ul class="typeahead dropdown-menu"></ul>',item:'<li><a href="#"></a></li>',minLength:1},a.fn.typeahead.Constructor=b,a.fn.typeahead.noConflict=function(){return a.fn.typeahead=c,this},a(document).on("focus.typeahead.data-api",'[data-provide="typeahead"]',function(b){var c=a(this);if(c.data("typeahead"))return;c.typeahead(c.data())})}(window.jQuery)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/AQLTS/queries/3.aql b/asterix-app/src/test/resources/AQLTS/queries/3.aql
deleted file mode 100644
index 3d71b27..0000000
--- a/asterix-app/src/test/resources/AQLTS/queries/3.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-for $current_sig in dataset('SIGroup')
-where
-   every $old_sig in dataset('SIGroup', getCurrentDateTime() - dtduration(0, 24, 0, 0))
-   satisfies $old_sig.name != $current_sig.name
-return $current_sig
diff --git a/asterix-app/src/test/resources/hadoop/conf/mapred-site.xml b/asterix-app/src/test/resources/hadoop/conf/mapred-site.xml
index 1b9a4d6..b39fced 100644
--- a/asterix-app/src/test/resources/hadoop/conf/mapred-site.xml
+++ b/asterix-app/src/test/resources/hadoop/conf/mapred-site.xml
@@ -5,21 +5,21 @@
 
 <configuration>
 
-  <property>
-    <name>mapred.job.tracker</name>
-    <value>localhost:29007</value>
-  </property>
-  <property>
-     <name>mapred.tasktracker.map.tasks.maximum</name>
-     <value>20</value>
-  </property>
-   <property>
-      <name>mapred.tasktracker.reduce.tasks.maximum</name>
-      <value>20</value>
-   </property>
-   <property>
-      <name>mapred.min.split.size</name>
-      <value>65536</value>
-   </property>
+	<property>
+		<name>mapred.job.tracker</name>
+		<value>localhost:29007</value>
+	</property>
+	<property>
+		<name>mapred.tasktracker.map.tasks.maximum</name>
+		<value>20</value>
+	</property>
+	<property>
+		<name>mapred.tasktracker.reduce.tasks.maximum</name>
+		<value>20</value>
+	</property>
+	<property>
+		<name>mapred.max.split.size</name>
+		<value>128</value>
+	</property>
 
 </configuration>
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.1.ddl.aql
new file mode 100644
index 0000000..73d63c9
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Description  : create a dataset using year-month-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:year-month-duration,
+dur:year-month-duration,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
+create index TestSecondIndex on Employee(dur);
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.2.update.aql
new file mode 100644
index 0000000..bd96595
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using year-month-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":year-month-duration("P16Y"), "dur":year-month-duration("-P23Y"), "name": "John"})
+insert into dataset Employee({"id":year-month-duration("-P37M"), "dur":year-month-duration("P1Y48M"), "name": "Alex"})
+insert into dataset Employee({"id":year-month-duration("P2013Y"), "dur":year-month-duration("P7M"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.3.query.aql
new file mode 100644
index 0000000..8244c6d
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Description  : create a dataset using year-month-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.dur > year-month-duration("P1Y")
+return $x
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.1.ddl.aql
new file mode 100644
index 0000000..9697c32
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Description  : create a dataset using datetime for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:datetime,
+dt:datetime,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
+create index TestSecondIndex on Employee(dt);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.2.update.aql
new file mode 100644
index 0000000..d66da92
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using datetime for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":datetime("1900-01-01T00:00:00"), "dt":datetime("1900-01-01T00:00:00"), "name": "John"})
+insert into dataset Employee({"id":datetime("2000-01-01T00:00:00"), "dt":datetime("2000-01-01T00:00:00"), "name": "Alex"})
+insert into dataset Employee({"id":datetime("2013-01-01T00:00:00"), "dt":datetime("2013-01-01T00:00:00"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.3.query.aql
new file mode 100644
index 0000000..eb92aea
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using datetime for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.dt > datetime("2007-07-07T07:07:07.777Z")
+return $x
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.1.ddl.aql
new file mode 100644
index 0000000..d3c19f4
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Description  : create a dataset using time for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:time,
+tm:time,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
+create index TestSecondIndex on Employee(tm);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.2.update.aql
new file mode 100644
index 0000000..6f44d6c
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using time for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":time("03:10:00.493Z"), "tm":time("03:10:00.493Z"), "name": "John"})
+insert into dataset Employee({"id":time("20:37:19+08:00"), "tm":time("20:37:19+08:00"), "name": "Alex"})
+insert into dataset Employee({"id":time("21:39:17.948-04:00"), "tm":time("21:39:17.948-04:00"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.3.query.aql
new file mode 100644
index 0000000..f0392b1
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using time for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.tm > time("07:07:07.777Z")
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.1.ddl.aql
new file mode 100644
index 0000000..ab88978
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Description  : create a dataset using date for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:date,
+dt:date,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
+create index TestSecondIndex on Employee(dt);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.2.update.aql
new file mode 100644
index 0000000..b8d415e
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using date for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":date("2010-01-01"), "dt":date("2010-01-01"), "name": "John"})
+insert into dataset Employee({"id":date("-1912-10-11"), "dt":date("-1912-10-11"), "name": "Alex"})
+insert into dataset Employee({"id":date("0732-02-02"), "dt":date("0732-02-02"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.3.query.aql
new file mode 100644
index 0000000..128a2a6
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using date for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.dt > date("2007-07-07")
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.1.ddl.aql
new file mode 100644
index 0000000..a935975
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Description  : create a dataset using day-time-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:day-time-duration,
+dur:day-time-duration,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
+create index TestSecondIndex on Employee(dur);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.2.update.aql
new file mode 100644
index 0000000..d5aafe9
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using day-time-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":day-time-duration("P380DT983M"), "dur":day-time-duration("P380DT983M"), "name": "John"})
+insert into dataset Employee({"id":day-time-duration("-P3829H849.392S"), "dur":day-time-duration("-P3829H849.392S"), "name": "Alex"})
+insert into dataset Employee({"id":day-time-duration("PT93847M0.392S"), "dur":day-time-duration("PT93847M0.392S"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.3.query.aql
new file mode 100644
index 0000000..dbca351
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using day-time-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+ 
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.dur > day-time-duration("P350D")
+return $x
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_0/issue363_temporal_sec_key_0.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_0/issue363_temporal_sec_key_0.1.adm
new file mode 100644
index 0000000..f1a3b14
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_0/issue363_temporal_sec_key_0.1.adm
@@ -0,0 +1 @@
+{ "id": year-month-duration("-P3Y1M"), "dur": year-month-duration("P5Y"), "name": "Alex" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_1/issue363_temporal_sec_key_1.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_1/issue363_temporal_sec_key_1.1.adm
new file mode 100644
index 0000000..1703ccd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_1/issue363_temporal_sec_key_1.1.adm
@@ -0,0 +1 @@
+{ "id": datetime("2013-01-01T00:00:00.000Z"), "dt": datetime("2013-01-01T00:00:00.000Z"), "name": "Bob" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_2/issue363_temporal_sec_key_2.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_2/issue363_temporal_sec_key_2.1.adm
new file mode 100644
index 0000000..1e32e45
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_2/issue363_temporal_sec_key_2.1.adm
@@ -0,0 +1,2 @@
+{ "id": time("12:37:19.000Z"), "tm": time("12:37:19.000Z"), "name": "Alex" }
+{ "id": time("01:39:17.948Z"), "tm": time("01:39:17.948Z"), "name": "Bob" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_3/issue363_temporal_sec_key_3.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_3/issue363_temporal_sec_key_3.1.adm
new file mode 100644
index 0000000..fb4a286
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_3/issue363_temporal_sec_key_3.1.adm
@@ -0,0 +1 @@
+{ "id": date("2010-01-01"), "dt": date("2010-01-01"), "name": "John" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_4/issue363_temporal_sec_key_4.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_4/issue363_temporal_sec_key_4.1.adm
new file mode 100644
index 0000000..dacaf15
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_4/issue363_temporal_sec_key_4.1.adm
@@ -0,0 +1 @@
+{ "id": day-time-duration("P380DT16H23M"), "dur": day-time-duration("P380DT16H23M"), "name": "John" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/testsuite.xml b/asterix-app/src/test/resources/metadata/testsuite.xml
index 63487d4..e9ed34d 100644
--- a/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -175,6 +175,31 @@
         <output-dir compare="Text">issue_363_temporal_key_4</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_sec_key_0">
+        <output-dir compare="Text">issue_363_temporal_sec_key_0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_sec_key_1">
+        <output-dir compare="Text">issue_363_temporal_sec_key_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_sec_key_2">
+        <output-dir compare="Text">issue_363_temporal_sec_key_2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_sec_key_3">
+        <output-dir compare="Text">issue_363_temporal_sec_key_3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_sec_key_4">
+        <output-dir compare="Text">issue_363_temporal_sec_key_4</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="exception">
     <test-case FilePath="exception">
diff --git a/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan b/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan
index 494b208..76fd0bd 100644
--- a/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan
@@ -11,11 +11,11 @@
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                     }
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$23(ASC), $$6(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$23(ASC), $$4(ASC)]  |PARTITIONED|
                   -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$1][$$6]  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$1][$$4]  |PARTITIONED|
                           -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- UNNEST  |PARTITIONED|
@@ -25,7 +25,7 @@
                                       -- DATASOURCE_SCAN  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$4]  |PARTITIONED|
                             -- RUNNING_AGGREGATE  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- SORT_MERGE_EXCHANGE [$$24(DESC) ]  |PARTITIONED|
@@ -37,13 +37,13 @@
                                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                                               }
                                         -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$30(ASC)] HASH:[$$30]  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$5]  |PARTITIONED|
+                                          -- PRE_CLUSTERED_GROUP_BY[$$3]  |PARTITIONED|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                   }
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$5(ASC)]  |PARTITIONED|
+                                              -- STABLE_SORT [$$3(ASC)]  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                     -- UNNEST  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan b/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan
index 0713f96..0c7b95d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan
@@ -28,7 +28,7 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$57][$$4]  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$57][$$3]  |PARTITIONED|
                                               -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -47,7 +47,7 @@
                                                               -- DATASOURCE_SCAN  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$4]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- STREAM_SELECT  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
index 84bc4de..d7c4c08 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
index d7a4c06..1f53ded 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
index d7a4c06..1f53ded 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
index 021f810..8a40be1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
index aec97d2..f26c81d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
index 2a2ccc4..7ad0078 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
index 2a2ccc4..7ad0078 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
index 7e8a594..7fcefa6 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
index 7e8a594..7fcefa6 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
index 5eef58e..dca0403 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
index d7a4c06..1f53ded 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
index aec97d2..f26c81d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
index aec97d2..f26c81d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
index d7a4c06..1f53ded 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
index aec97d2..f26c81d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
index aec97d2..f26c81d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
index aec97d2..f26c81d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
index 2a2ccc4..7ad0078 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
index 2a2ccc4..7ad0078 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
index 3028f9a..a59363b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
index 3028f9a..a59363b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
index aec97d2..f26c81d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
index 78e761f..e4516f1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
index 57730fe..6900fe8 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
@@ -13,7 +13,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                              -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
index a1a1253..e62367e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                          -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
index 2a2ccc4..7ad0078 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
index 2a2ccc4..7ad0078 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
index 78e761f..e4516f1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                      -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
index 57730fe..6900fe8 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
@@ -13,7 +13,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                              -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
index 0fab288..c683c43 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
@@ -15,7 +15,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                  -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
                                       -- STREAM_SELECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance.plan
index f7d3cb4..0aaa618 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance.plan
@@ -14,7 +14,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
index 1be3fb1..fc5a042 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
@@ -14,7 +14,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.plan
index 7c74f83..7629035 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
index c5534d6..4972d0f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
@@ -13,7 +13,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                              -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard.plan
index fdbf50f..a8c2925 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
index 0fab288..c683c43 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
@@ -15,7 +15,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                  -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
                                       -- STREAM_SELECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance.plan
index f7d3cb4..0aaa618 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance.plan
@@ -14,7 +14,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.plan
index 1be3fb1..fc5a042 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.plan
@@ -14,7 +14,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.plan
index 88c4469..a2236b0 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
index 8b6b08b..ebebe4d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
@@ -13,7 +13,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                              -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard.plan
index ba9879d..0ee49d0 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.plan
index 88c4469..a2236b0 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
index 8b6b08b..ebebe4d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
@@ -13,7 +13,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                              -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard.plan
index ba9879d..0ee49d0 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.plan
index 7c74f83..7629035 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
index c5534d6..4972d0f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
@@ -13,7 +13,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                              -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard.plan
index fdbf50f..a8c2925 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                            -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan
index a04e378..b274dfc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
index 1800e75..9fe499b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
index a04e378..b274dfc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
index abd2c39..dbeabdd 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
@@ -22,7 +22,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                        -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
                                             -- STREAM_SELECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan
index a04e378..b274dfc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
index 1800e75..9fe499b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
index a04e378..b274dfc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
index dffe718..7a789d2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
@@ -22,7 +22,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                        -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
                                             -- STREAM_SELECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
index 65be14f..16f2bfd 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan
index bfee1a5..595587e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
index bfee1a5..595587e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan
index ab5cf83..b7fe0db 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
index 7877b30..7160d6f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
index ab5cf83..b7fe0db 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan
index 3f7ecb1..2710b67 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
index 4b767ab..c3bf90e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
index 3f7ecb1..2710b67 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
index 77b00ff..5ed5498 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan
index 3f7ecb1..2710b67 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
index 4b767ab..c3bf90e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
index 3f7ecb1..2710b67 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
index 32d7e09..c995dafa 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan
index a04e378..b274dfc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
index 1800e75..9fe499b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
index a04e378..b274dfc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
index abd2c39..dbeabdd 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
@@ -22,7 +22,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                        -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
                                             -- STREAM_SELECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan
index a04e378..b274dfc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
index 1800e75..9fe499b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
index a04e378..b274dfc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
index dffe718..7a789d2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
@@ -22,7 +22,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                        -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
                                             -- STREAM_SELECT  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan
index bfee1a5..595587e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
index 65be14f..16f2bfd 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
index bfee1a5..595587e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan
index 2a4e9de..74bdea2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
index 06826c1..be387f0 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
index 2a4e9de..74bdea2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan
index 7c16cef..dc2073c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
index 3d0f3f1..8a7e71b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
index 7c16cef..dc2073c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
index 156dd43..4934b82 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan
index 7c16cef..dc2073c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
index 3d0f3f1..8a7e71b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
index 7c16cef..dc2073c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
index 47f5c91..710ba98 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan
index 2a4e9de..74bdea2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
index 06826c1..be387f0 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
index 2a4e9de..74bdea2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan
index 7c16cef..dc2073c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
index 3d0f3f1..8a7e71b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
index 7c16cef..dc2073c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
index 156dd43..4934b82 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan
index 7c16cef..dc2073c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
index 3d0f3f1..8a7e71b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
index 7c16cef..dc2073c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
index 47f5c91..710ba98 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan
index ab5cf83..b7fe0db 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
index 7877b30..7160d6f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
index ab5cf83..b7fe0db 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan
index 3f7ecb1..2710b67 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
index 0fced26..e58089b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
index 3f7ecb1..2710b67 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
index 77b00ff..5ed5498 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan
index 3f7ecb1..2710b67 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
index 4b767ab..c3bf90e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
index 3f7ecb1..2710b67 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
index 32d7e09..c995dafa 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                    -- FUZZY_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
index e969e5d..cba987c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
@@ -3,8 +3,8 @@
     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
       -- INSERT_DELETE  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
-            -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
index 4f0d931..e1852af 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
@@ -10,8 +10,8 @@
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- INSERT_DELETE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+                        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
@@ -20,4 +20,4 @@
                                       -- BTREE_SEARCH  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
index 0608f69..a5265c8 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
@@ -3,8 +3,8 @@
     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
       -- INSERT_DELETE  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-            -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
index 1458c66..8057666 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
@@ -13,8 +13,8 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- INSERT_DELETE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$6(ASC)]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
+                              -- STABLE_SORT [$$7(ASC)]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$7]  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
index 0b756d9..51d3060 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
@@ -3,8 +3,8 @@
     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
       -- INSERT_DELETE  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$6(ASC)]  |PARTITIONED|
-            -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
+          -- STABLE_SORT [$$7(ASC)]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$7]  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.3.query.aql
index 9819455..4611058 100644
--- a/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.3.query.aql
@@ -6,4 +6,4 @@
  */
  
 let $l := [[1,2,3,4,5],[6,7,8,9]]
-return count(for $i in $l return $i)
\ No newline at end of file
+return count(for $i in $l return $i)
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql
deleted file mode 100644
index d00347d..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql
+++ /dev/null
@@ -1,24 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
- * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index that are built on nullable fields. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string?,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql
deleted file mode 100644
index 1d5d4fd..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql
+++ /dev/null
@@ -1,12 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
- * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index that are built on nullable fields. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql
deleted file mode 100644
index a79828c..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
- * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index that are built on nullable fields. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-create index fuzzy_ngram_index on DBLP(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql
deleted file mode 100644
index 7e917b5..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
- * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index that are built on nullable fields.
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-delete $o from dataset DBLP where $o.id>50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql
deleted file mode 100644
index c1c6247..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql
+++ /dev/null
@@ -1,13 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
- * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index that are built on nullable fields.
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql
deleted file mode 100644
index e98b328..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql
+++ /dev/null
@@ -1,23 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index.aql
- * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index.
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.2.update.aql
deleted file mode 100644
index 381ce87..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.2.update.aql
+++ /dev/null
@@ -1,12 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index.aql
- * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index.
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql
deleted file mode 100644
index ede69f6..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index.aql
- * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index.
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-create index fuzzy_ngram_index on DBLP(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.4.update.aql
deleted file mode 100644
index 9c58c7e..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.4.update.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index.aql
- * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index.
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-delete $o from dataset DBLP where $o.id>50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.5.query.aql
deleted file mode 100644
index ca070fc..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.5.query.aql
+++ /dev/null
@@ -1,13 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index.aql
- * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index.
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql
deleted file mode 100644
index 45570ac..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql
+++ /dev/null
@@ -1,24 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.aql
- * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index that are built on nullable fields. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string?,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql
deleted file mode 100644
index d26c270..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql
+++ /dev/null
@@ -1,12 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.aql
- * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index that are built on nullable fields. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql
deleted file mode 100644
index 3747bb1..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.aql
- * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index that are built on nullable fields. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-create index fuzzy_keyword_index on DBLP(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql
deleted file mode 100644
index eb15450..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.aql
- * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index that are built on nullable fields. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-delete $o from dataset DBLP where $o.id<50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql
deleted file mode 100644
index c1fb278..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql
+++ /dev/null
@@ -1,14 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.aql
- * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index that are built on nullable fields. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-for $o in dataset('DBLP')
-let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
-where $jacc[0]
-return $o
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.1.ddl.aql
deleted file mode 100644
index 9d04c14..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.1.ddl.aql
+++ /dev/null
@@ -1,24 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index.aql
- * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index.
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.2.update.aql
deleted file mode 100644
index a5995f6..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.2.update.aql
+++ /dev/null
@@ -1,12 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index.aql
- * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index.
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.3.ddl.aql
deleted file mode 100644
index 97aef53..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.3.ddl.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index.aql
- * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index.
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-create index fuzzy_keyword_index on DBLP(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.4.update.aql
deleted file mode 100644
index 001e605..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.4.update.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index.aql
- * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index.
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-delete $o from dataset DBLP where $o.id<50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.5.query.aql
deleted file mode 100644
index 7910859..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.5.query.aql
+++ /dev/null
@@ -1,14 +0,0 @@
-/* 
- * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index.aql
- * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index.
- * Expected Result : Success
- * Date            : March 31 2013
- */
-
-use dataverse test;
-
-for $o in dataset('DBLP')
-let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
-where $jacc[0]
-return $o
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql
deleted file mode 100644
index 21ff64a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
- * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index that are built on nullable fields.  
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string?,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-create dataset DBLP1(DBLPType) primary key id;
-
-
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql
deleted file mode 100644
index 9108ad6..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql
+++ /dev/null
@@ -1,12 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
- * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index that are built on nullable fields.  
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql
deleted file mode 100644
index 426654c..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql
+++ /dev/null
@@ -1,11 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
- * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index that are built on nullable fields.  
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-create index fuzzy_ngram_index on DBLP(title) type fuzzy ngram(3);
-create index fuzzy_ngram_index1 on DBLP1(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql
deleted file mode 100644
index 2fe30df..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql
+++ /dev/null
@@ -1,21 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
- * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index that are built on nullable fields.  
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-insert into dataset DBLP1 (
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return {
-		"id": $o.id,
-		"dblpid": $o.dblpid,
-		"title": $o.title,
-		"authors": $o.authors,
-		"misc": $o.misc
-	}
-);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql
deleted file mode 100644
index f61df2a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql
+++ /dev/null
@@ -1,13 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
- * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index that are built on nullable fields.  
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-for $o in dataset('DBLP1')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql
deleted file mode 100644
index af890c4..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index.aql
- * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-create dataset DBLP1(DBLPType) primary key id;
-
-
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.2.update.aql
deleted file mode 100644
index 382a4a6..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.2.update.aql
+++ /dev/null
@@ -1,12 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index.aql
- * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql
deleted file mode 100644
index 8109213..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql
+++ /dev/null
@@ -1,11 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index.aql
- * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-create index fuzzy_ngram_index on DBLP(title) type fuzzy ngram(3);
-create index fuzzy_ngram_index1 on DBLP1(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.4.update.aql
deleted file mode 100644
index b829b15..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.4.update.aql
+++ /dev/null
@@ -1,21 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index.aql
- * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-insert into dataset DBLP1 (
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return {
-		"id": $o.id,
-		"dblpid": $o.dblpid,
-		"title": $o.title,
-		"authors": $o.authors,
-		"misc": $o.misc
-	}
-);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.5.query.aql
deleted file mode 100644
index 324b351..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.5.query.aql
+++ /dev/null
@@ -1,13 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index.aql
- * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-for $o in dataset('DBLP1')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql
deleted file mode 100644
index 7dbf7ee..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.aql
- * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index that are built on nullable fields.  
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string?,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-create dataset DBLP1(DBLPType) primary key id;
-
-
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql
deleted file mode 100644
index 1d8126f..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql
+++ /dev/null
@@ -1,12 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.aql
- * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index that are built on nullable fields.  
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql
deleted file mode 100644
index 798622a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql
+++ /dev/null
@@ -1,11 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.aql
- * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index that are built on nullable fields.  
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-create index fuzzy_keyword_index on DBLP(title) type fuzzy keyword;
-create index fuzzy_keyword_index1 on DBLP1(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql
deleted file mode 100644
index 3234c4e..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql
+++ /dev/null
@@ -1,20 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.aql
- * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index that are built on nullable fields.  
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-insert into dataset DBLP1 (
-for $o in dataset('DBLP')
-order by $o.id
-return {
-		"id": $o.id,
-		"dblpid": $o.dblpid,
-		"title": $o.title,
-		"authors": $o.authors,
-		"misc": $o.misc
-	}
-);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql
deleted file mode 100644
index 43ad95c..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql
+++ /dev/null
@@ -1,14 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.aql
- * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index that are built on nullable fields.  
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-for $o in dataset('DBLP1')
-let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
-where $jacc[0]
-return $o
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.1.ddl.aql
deleted file mode 100644
index 5dd6f1a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.1.ddl.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index.aql
- * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-create dataset DBLP1(DBLPType) primary key id;
-
-
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.2.update.aql
deleted file mode 100644
index 25e9446..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.2.update.aql
+++ /dev/null
@@ -1,12 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index.aql
- * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.3.ddl.aql
deleted file mode 100644
index f43c5cf..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.3.ddl.aql
+++ /dev/null
@@ -1,11 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index.aql
- * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-create index fuzzy_keyword_index on DBLP(title) type fuzzy keyword;
-create index fuzzy_keyword_index1 on DBLP1(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.4.update.aql
deleted file mode 100644
index b8a1af1..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.4.update.aql
+++ /dev/null
@@ -1,20 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index.aql
- * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-insert into dataset DBLP1 (
-for $o in dataset('DBLP')
-order by $o.id
-return {
-		"id": $o.id,
-		"dblpid": $o.dblpid,
-		"title": $o.title,
-		"authors": $o.authors,
-		"misc": $o.misc
-	}
-);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.5.query.aql
deleted file mode 100644
index 0fe611a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.5.query.aql
+++ /dev/null
@@ -1,14 +0,0 @@
-/* 
- * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index.aql
- * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index. 
- * Expected Result : Success
- * Date            : March 31 2013
- */
- 
-use dataverse test;
-
-for $o in dataset('DBLP1')
-let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
-where $jacc[0]
-return $o
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/opentype/opentype.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/opentype/opentype.1.ddl.aql
new file mode 100644
index 0000000..17da24b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/opentype/opentype.1.ddl.aql
@@ -0,0 +1,13 @@
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+use dataverse TinySocial;
+
+create type TweetMessageType as open {
+        tweetid: string
+}
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid
+hints(cardinality=100); 
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/opentype/opentype.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/opentype/opentype.2.update.aql
new file mode 100644
index 0000000..627623a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/opentype/opentype.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse TinySocial;
+
+load dataset TweetMessages
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/tinysocial/twm.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/opentype/opentype.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/opentype/opentype.3.query.aql
new file mode 100644
index 0000000..f40e884
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/opentype/opentype.3.query.aql
@@ -0,0 +1,15 @@
+use dataverse TinySocial;
+
+set simfunction "jaccard";
+set simthreshold "0.3";
+
+for $t in dataset TweetMessages
+order by $t.tweetid
+return {                     
+    "tweet": $t,               
+    "similar-tweets": for $t2 in dataset TweetMessages
+    					order by $t2.tweetid
+                        where  $t2.referred-topics ~= $t.referred-topics
+                        and $t2.tweetid != $t.tweetid
+                        return $t2.referred-topics
+};
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/hdfs/issue_245_hdfs/issue_245_hdfs.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/hdfs/issue_245_hdfs/issue_245_hdfs.3.query.aql
index 653ee6c..527a0e5 100644
--- a/asterix-app/src/test/resources/runtimets/queries/hdfs/issue_245_hdfs/issue_245_hdfs.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/hdfs/issue_245_hdfs/issue_245_hdfs.3.query.aql
@@ -8,4 +8,5 @@
 use dataverse test;
 
 for $x in dataset('TextDataset')
+order by $x.line
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.1.ddl.aql
deleted file mode 100644
index d9ca825..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.1.ddl.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-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 nodegroup group1 if not exists on nc1, nc2;
-
-create dataset DBLP(DBLPType) 
-  primary key id on group1;
-
-create index ngram_index on DBLP(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.2.update.aql
deleted file mode 100644
index 830e08a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.2.update.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-use dataverse test;
-
-load dataset DBLP 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.3.query.aql
deleted file mode 100644
index 866b045..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.3.query.aql
+++ /dev/null
@@ -1,6 +0,0 @@
-use dataverse test;
-
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.1.ddl.aql
deleted file mode 100644
index 7afd29a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.1.ddl.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-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 nodegroup group1 if not exists on nc1, nc2;
-
-create dataset DBLP(DBLPType) 
-  primary key id on group1;
-
-create index ngram_index on DBLP(authors) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.2.update.aql
deleted file mode 100644
index 830e08a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.2.update.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-use dataverse test;
-
-load dataset DBLP 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.3.query.aql
deleted file mode 100644
index 45ac926..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.3.query.aql
+++ /dev/null
@@ -1,6 +0,0 @@
-use dataverse test;
-
-for $o in dataset('DBLP')
-let $ed := edit-distance-check($o.authors, "Amihay Motro", 5)
-where $ed[0]
-return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.1.ddl.aql
deleted file mode 100644
index 7afd29a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.1.ddl.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-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 nodegroup group1 if not exists on nc1, nc2;
-
-create dataset DBLP(DBLPType) 
-  primary key id on group1;
-
-create index ngram_index on DBLP(authors) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.2.update.aql
deleted file mode 100644
index 830e08a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.2.update.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-use dataverse test;
-
-load dataset DBLP 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.3.query.aql
deleted file mode 100644
index 0f4c003..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.3.query.aql
+++ /dev/null
@@ -1,6 +0,0 @@
-use dataverse test;
-
-for $o in dataset('DBLP')
-let $ed := edit-distance-check($o.authors, "Amihay Motro", 1)
-where $ed[0]
-return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.1.ddl.aql
deleted file mode 100644
index d9ca825..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.1.ddl.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-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 nodegroup group1 if not exists on nc1, nc2;
-
-create dataset DBLP(DBLPType) 
-  primary key id on group1;
-
-create index ngram_index on DBLP(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.2.update.aql
deleted file mode 100644
index 830e08a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.2.update.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-use dataverse test;
-
-load dataset DBLP 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.3.query.aql
deleted file mode 100644
index dc18182..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.3.query.aql
+++ /dev/null
@@ -1,7 +0,0 @@
-use dataverse test;
-set import-private-functions 'true';
-
-for $o in dataset('DBLP')
-let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
-where $jacc[0]
-return $o
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.1.ddl.aql
deleted file mode 100644
index 8a57319..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.1.ddl.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create nodegroup group1 if not exists on nc1;
-
-create dataset Customers(CustomerType) 
-  primary key cid on group1;
-
-create index interests_index on Customers(interests) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.2.update.aql
deleted file mode 100644
index 063b172..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.2.update.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-use dataverse test;
-
-load dataset Customers 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.3.query.aql
deleted file mode 100644
index d4a4f60..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.3.query.aql
+++ /dev/null
@@ -1,7 +0,0 @@
-use dataverse test;
-
-for $c in dataset('Customers')
-let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)
-where $ed[0]
-order by $c.cid
-return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.1.ddl.aql
deleted file mode 100644
index 8a57319..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.1.ddl.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create nodegroup group1 if not exists on nc1;
-
-create dataset Customers(CustomerType) 
-  primary key cid on group1;
-
-create index interests_index on Customers(interests) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.2.update.aql
deleted file mode 100644
index 063b172..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.2.update.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-use dataverse test;
-
-load dataset Customers 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.3.query.aql
deleted file mode 100644
index 7ebd73e..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.3.query.aql
+++ /dev/null
@@ -1,7 +0,0 @@
-use dataverse test;
-
-for $c in dataset('Customers')
-let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)
-where $ed[0]
-order by $c.cid
-return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.1.ddl.aql
deleted file mode 100644
index 8a57319..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.1.ddl.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create nodegroup group1 if not exists on nc1;
-
-create dataset Customers(CustomerType) 
-  primary key cid on group1;
-
-create index interests_index on Customers(interests) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.2.update.aql
deleted file mode 100644
index 063b172..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.2.update.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-use dataverse test;
-
-load dataset Customers 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.3.query.aql
deleted file mode 100644
index 27a838ba..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.3.query.aql
+++ /dev/null
@@ -1,6 +0,0 @@
-use dataverse test;
-
-for $c in dataset('Customers')
-let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
-where $jacc[0]
-return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.1.ddl.aql
deleted file mode 100644
index 3e2562a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.1.ddl.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create nodegroup group1 if not exists on nc1;
-
-create dataset Customers(CustomerType) 
-  primary key cid on group1;
-
-create index interests_index on Customers(interests) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.2.update.aql
deleted file mode 100644
index 0deeabc..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.2.update.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-use dataverse test;
-
-load dataset Customers 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.3.query.aql
deleted file mode 100644
index 27a838ba..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.3.query.aql
+++ /dev/null
@@ -1,6 +0,0 @@
-use dataverse test;
-
-for $c in dataset('Customers')
-let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
-where $jacc[0]
-return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.1.ddl.aql
deleted file mode 100644
index 326b264..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.1.ddl.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-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 nodegroup group1 if not exists on nc1, nc2;
-
-create dataset DBLP(DBLPType) 
-  primary key id on group1;
-
-create index keyword_index on DBLP(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.2.update.aql
deleted file mode 100644
index 830e08a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.2.update.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-use dataverse test;
-
-load dataset DBLP 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.3.query.aql
deleted file mode 100644
index d6de89d..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.3.query.aql
+++ /dev/null
@@ -1,6 +0,0 @@
-use dataverse test;
-
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.1.ddl.aql
deleted file mode 100644
index 326b264..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.1.ddl.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-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 nodegroup group1 if not exists on nc1, nc2;
-
-create dataset DBLP(DBLPType) 
-  primary key id on group1;
-
-create index keyword_index on DBLP(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.2.update.aql
deleted file mode 100644
index 830e08a..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.2.update.aql
+++ /dev/null
@@ -1,5 +0,0 @@
-use dataverse test;
-
-load dataset DBLP 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.3.query.aql
deleted file mode 100644
index 7ce908c..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.3.query.aql
+++ /dev/null
@@ -1,7 +0,0 @@
-use dataverse test;
-
-for $o in dataset('DBLP')
-let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
-where $jacc[0]
-return $o
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.1.ddl.aql
new file mode 100644
index 0000000..305a2fc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.1.ddl.aql
@@ -0,0 +1,26 @@
+drop dataverse DMLTest if exists;
+create dataverse DMLTest;
+use dataverse DMLTest;
+
+create type EmploymentType as closed {
+organization-name: string,
+start-date: date,
+end-date: date?
+}
+
+create type FacebookUserType as closed {
+id: int32,
+id-copy: int32,
+alias: string,
+name: string,
+user-since: datetime,
+user-since-copy: datetime,
+friend-ids: {{ int32 }},
+employment: [EmploymentType]
+}
+
+create dataset FacebookUsers1(FacebookUserType)
+primary key id;
+
+create dataset FacebookUsers2(FacebookUserType)
+primary key id;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.2.update.aql
new file mode 100644
index 0000000..7167f1d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse DMLTest;
+
+load dataset FacebookUsers1
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/fbu.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.3.update.aql b/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.3.update.aql
new file mode 100644
index 0000000..f19af86
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.3.update.aql
@@ -0,0 +1,6 @@
+use dataverse DMLTest;
+
+insert into dataset FacebookUsers2 (
+for $t in dataset FacebookUsers1
+return $t
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.4.query.aql
new file mode 100644
index 0000000..12e632f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/misc/flushtest/flushtest.4.query.aql
@@ -0,0 +1,4 @@
+use dataverse DMLTest;
+
+for $t in dataset('FacebookUsers2')
+return $t
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/caret0/caret0.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/caret0/caret0.1.ddl.aql
new file mode 100644
index 0000000..139597f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/caret0/caret0.1.ddl.aql
@@ -0,0 +1,2 @@
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/caret0/caret0.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/caret0/caret0.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/caret0/caret0.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/caret0/caret0.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/caret0/caret0.3.query.aql
new file mode 100644
index 0000000..825b269
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/caret0/caret0.3.query.aql
@@ -0,0 +1,5 @@
+let $n1 := 2.0
+let $n2 := 4096.0
+let $n3 := 3
+let $n4 := 2
+return { "c1": $n1^$n2, "c2": $n2^$n1, "c3": $n3^$n4 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/query-issue355/query-issue355.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/query-issue355/query-issue355.1.ddl.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/query-issue355/query-issue355.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/query-issue355/query-issue355.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/query-issue355/query-issue355.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/query-issue355/query-issue355.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/query-issue355/query-issue355.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/query-issue355/query-issue355.3.query.aql
new file mode 100644
index 0000000..599bc16
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/query-issue355/query-issue355.3.query.aql
@@ -0,0 +1,3 @@
+for $a in [{"name":"Bob","age":10,"sex":"Male"},{"name":"John","age":45,"sex":"Female"},{"name":"Raj","age":35,"sex":"Male"}]
+where string-length($a.name) > -10000000000000000000
+return [$a.age, string-equal(lowercase($a.name), "john")]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.3.query.aql
index 81d6cf6..02ff97d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.3.query.aql
@@ -11,6 +11,7 @@
 set simthreshold "3";
 
 for $fbu in dataset FacebookUsers
+order by $fbu.id
 return {
     "id": $fbu.id,
     "name": $fbu.name,
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423-2/query-issue423-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423-2/query-issue423-2.1.ddl.aql
new file mode 100644
index 0000000..d40df1b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423-2/query-issue423-2.1.ddl.aql
@@ -0,0 +1,15 @@
+/*
+ * Description  : This test case is to verify the fix for issue423
+ 				  (Specifically for NLJ case)
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=423
+ * Expected Res : Success
+ * Date         : 29th May 2013
+ */
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TestType as open { id : int32 ,fname:string, lname:string}
+
+create dataset t2(TestType) primary key fname,lname;
+create dataset t1(TestType) primary key fname,lname;
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423-2/query-issue423-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423-2/query-issue423-2.2.update.aql
new file mode 100644
index 0000000..ef89d7f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423-2/query-issue423-2.2.update.aql
@@ -0,0 +1,14 @@
+/*
+ * Description  : This test case is to verify the fix for issue423
+ 				  (Specifically for NLJ case)
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=423
+ * Expected Res : Success
+ * Date         : 29th May 2013
+ */
+ 
+use dataverse test;
+
+insert into dataset t1({"id":123,"fname":"John","lname":"Doe"});
+insert into dataset t1({"id":122,"fname":"Bruce","lname":"Li"});
+insert into dataset t2({"id":23,"fname":"John","lname":"Doe"});
+insert into dataset t2({"id":24,"fname":"Ravi","lname":"Khanna"});
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423-2/query-issue423-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423-2/query-issue423-2.3.query.aql
new file mode 100644
index 0000000..38e7729
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423-2/query-issue423-2.3.query.aql
@@ -0,0 +1,14 @@
+/*
+ * Description  : This test case is to verify the fix for issue423
+ 				  (Specifically for NLJ case)
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=423
+ * Expected Res : Success
+ * Date         : 29th May 2013
+ */
+ 
+use dataverse test;
+
+for $l in dataset t1
+for $m in dataset t2
+	where $l.age>$m.age
+return {"l":$l,"m":$m};
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.1.ddl.aql
index 05eb126..7491fcc 100644
--- a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.1.ddl.aql
@@ -1,3 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue423
+ 				  (Specifically for HHJ case)
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=423
+ * Expected Res : Success
+ * Date         : 29th May 2013
+ */
+ 
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.2.update.aql
index 71904f1..40ee53e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.2.update.aql
@@ -1,3 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue423
+ 				  (Specifically for HHJ case)
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=423
+ * Expected Res : Success
+ * Date         : 29th May 2013
+ */
+ 
 use dataverse test;
 
 insert into dataset t1({"id":123,"fname":"John","lname":"Doe"});
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.3.query.aql
index a07b185..d124385 100644
--- a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.3.query.aql
@@ -1,3 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue423
+ 				  (Specifically for HHJ case)
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=423
+ * Expected Res : Success
+ * Date         : 29th May 2013
+ */
+ 
 use dataverse test;
 
 for $l in dataset t1
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.1.ddl.aql
new file mode 100644
index 0000000..edd97cd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.1.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Description  : This test case is to verify the fix for issue456: 
+ *                https://code.google.com/p/asterixdb/issues/detail?id=456
+ * Expected Res : SUCCESS
+ * Date         : 3rd June 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TypeOpen as open {
+  id : int32,
+  int_m : int32,
+  int_o : int32?,
+  string_m : string,
+  string_o : string?
+};
+
+create dataset DataOpen(TypeOpen) primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.2.update.aql
new file mode 100644
index 0000000..e084b44
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * Description  : This test case is to verify the fix for issue456: 
+ *                https://code.google.com/p/asterixdb/issues/detail?id=456
+ * Expected Res : SUCCESS
+ * Date         : 3rd June 2013
+ */
+
+use dataverse test;
+
+insert into dataset DataOpen({ "id": 0, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": "a" });
+insert into dataset DataOpen({ "id": 1, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": null });
+insert into dataset DataOpen({ "id": 2, "int_m": 1, "int_o": 1, "string_m": "b", "string_o": "a" });
+insert into dataset DataOpen({ "id": 3, "int_m": 1, "int_o": 1, "string_m": "b", "string_o": null });
+insert into dataset DataOpen({ "id": 4, "int_m": 1, "int_o": null, "string_m": "a", "string_o": "a" });
+insert into dataset DataOpen({ "id": 5, "int_m": 1, "int_o": null, "string_m": "a", "string_o": null });
+insert into dataset DataOpen({ "id": 6, "int_m": 1, "int_o": null, "string_m": "b", "string_o": "a" });
+insert into dataset DataOpen({ "id": 7, "int_m": 1, "int_o": null, "string_m": "b", "string_o": null });
+insert into dataset DataOpen({ "id": 8, "int_m": 2, "int_o": 1, "string_m": "a", "string_o": "a" });
+insert into dataset DataOpen({ "id": 9, "int_m": 2, "int_o": 1, "string_m": "a", "string_o": null });
+insert into dataset DataOpen({ "id": 10, "int_m": 2, "int_o": 1, "string_m": "b", "string_o": "a" });
+insert into dataset DataOpen({ "id": 11, "int_m": 2, "int_o": 1, "string_m": "b", "string_o": null });
+insert into dataset DataOpen({ "id": 12, "int_m": 2, "int_o": null, "string_m": "a", "string_o": "a" });
+insert into dataset DataOpen({ "id": 13, "int_m": 2, "int_o": null, "string_m": "a", "string_o": null });
+insert into dataset DataOpen({ "id": 14, "int_m": 2, "int_o": null, "string_m": "b", "string_o": "a" });
+insert into dataset DataOpen({ "id": 15, "int_m": 2, "int_o": null, "string_m": "b", "string_o": null });
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.3.query.aql
new file mode 100644
index 0000000..217fbd1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Description  : This test case is to verify the fix for issue456: 
+ *                https://code.google.com/p/asterixdb/issues/detail?id=456
+ * Expected Res : SUCCESS
+ * Date         : 3rd June 2013
+ */
+
+use dataverse test;
+
+for $x in dataset DataOpen
+let $id := $x.id
+group by $m := $x.int_m with $id
+return [ $m, count($id) ]
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue465/query-issue465.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue465/query-issue465.1.ddl.aql
new file mode 100644
index 0000000..9f5fb23
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue465/query-issue465.1.ddl.aql
@@ -0,0 +1,7 @@
+/*
+ * Description  : This test case is to verify the fix for issue465: 
+ *                https://code.google.com/p/asterixdb/issues/detail?id=465
+ * Expected Res : SUCCESS
+ * Date         : 3rd June 2013
+ */
+ 
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue465/query-issue465.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue465/query-issue465.2.update.aql
new file mode 100644
index 0000000..9f5fb23
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue465/query-issue465.2.update.aql
@@ -0,0 +1,7 @@
+/*
+ * Description  : This test case is to verify the fix for issue465: 
+ *                https://code.google.com/p/asterixdb/issues/detail?id=465
+ * Expected Res : SUCCESS
+ * Date         : 3rd June 2013
+ */
+ 
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue465/query-issue465.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue465/query-issue465.3.query.aql
new file mode 100644
index 0000000..244b193
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue465/query-issue465.3.query.aql
@@ -0,0 +1,10 @@
+/*
+ * Description  : This test case is to verify the fix for issue465: 
+ *                https://code.google.com/p/asterixdb/issues/detail?id=465
+ * Expected Res : SUCCESS
+ * Date         : 3rd June 2013
+ */
+
+let $a := {"r1":1234}
+let $b := {"r2":456}
+return [ $a, $b ]
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue487/query-issue487.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue487/query-issue487.1.ddl.aql
new file mode 100644
index 0000000..4d74b83
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue487/query-issue487.1.ddl.aql
@@ -0,0 +1,18 @@
+/*
+ * Description  : This test case is to verify the fix for issue487
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=487
+ * Expected Res : FAIL
+ * Date         : 30th May 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type EmpType as open {
+id : int32,
+name : string
+}
+
+create dataset Employee(EmpType) primary key id;
+ 
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue487/query-issue487.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue487/query-issue487.2.update.aql
new file mode 100644
index 0000000..6913b4f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue487/query-issue487.2.update.aql
@@ -0,0 +1,10 @@
+/*
+ * Description  : This test case is to verify the fix for issue487
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=487
+ * Expected Res : FAIL
+ * Date         : 30th May 2013
+ */
+
+use dataverse test;
+ 
+insert into dataset Employee ({ "id":123});
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue487/query-issue487.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue487/query-issue487.3.query.aql
new file mode 100644
index 0000000..d53aba9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue487/query-issue487.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue487
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=487
+ * Expected Res : FAIL
+ * Date         : 30th May 2013
+ */
+
+use dataverse test;
+
+for $l in dataset Employee
+return $l;
+ 
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint2/string-to-codepoint2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint2/string-to-codepoint2.1.ddl.aql
new file mode 100644
index 0000000..754ea81
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint2/string-to-codepoint2.1.ddl.aql
@@ -0,0 +1,3 @@
+drop dataverse test if exists;
+create dataverse test;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint2/string-to-codepoint2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint2/string-to-codepoint2.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint2/string-to-codepoint2.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint2/string-to-codepoint2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint2/string-to-codepoint2.3.query.aql
new file mode 100644
index 0000000..88f0dfa
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-to-codepoint2/string-to-codepoint2.3.query.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+let $x := "欢迎"
+let $c := string-to-codepoint($x)
+
+return {"result1": $c}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.1.ddl.aql
index 280e52c..d595848 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.1.ddl.aql
@@ -1,6 +1,6 @@
 /*
  * Testcase Name  : substr04.aql
- * Description    : Test substring2(string,position,position) built in function.
+ * Description    : Test substring(string,position,position) built in function.
  * Success        : Yes
  * Date           : 18th April 2012
  */
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.2.update.aql
index 280e52c..d595848 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.2.update.aql
@@ -1,6 +1,6 @@
 /*
  * Testcase Name  : substr04.aql
- * Description    : Test substring2(string,position,position) built in function.
+ * Description    : Test substring(string,position,position) built in function.
  * Success        : Yes
  * Date           : 18th April 2012
  */
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
index 0f5b1ff..69ff8ab 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
@@ -1,19 +1,17 @@
 /*
  * Testcase Name  : substr04.aql
- * Description    : Test substring2(string,position,position) built in function.
+ * Description    : Test substring(string, position, length) built in function.
  * Success        : Yes
  * Date           : 18th April 2012
  */
 
-for $a in [ substring2("hello world",7,11),
-substring("hello world",1,11),
-substring("hello world",3,7),
-substring("ABCD",3,6),
-substring("ABCD",0,4),
-substring("UC Irvine",4,string-length("UC Irvine")),
-substring("UC Irvine",0,string-length("UC Irvine")),
-substring("UC Irvine",1,string-length("UC Irvine")),
-substring(substring("UC Irvine",4),0,string-length("Irvine")),
-substring(substring("UC Irvine",4),0,(string-length("Irvine")/2))
+for $a in [ substring("hello world", 7, 5),
+substring("hello world", 1, 11),
+substring("hello world", 3, 9),
+substring("ABCD", 3, 2),
+substring("ABCD", 1, 4),
+substring("UC Irvine", 4, string-length("UC Irvine") - 3),
+substring("UC Irvine", 1, string-length("UC Irvine")),
+substring(substring("UC Irvine", 4), 1, string-length("Irvine"))
 ]
 return $a
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.1.ddl.aql
index cd3bee1..6a77034 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.1.ddl.aql
@@ -1,6 +1,6 @@
 /*
  * Testcase Name  : substr05.aql
- * Description    : Test substring2(string,position,position) built in function.
+ * Description    : Test substring(string,position,position) built in function.
  * Success        : Yes
  * Date           : 19th April 2012
  */
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.2.update.aql
index da796c7..680f0db 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.2.update.aql
@@ -1,6 +1,6 @@
 /*
  * Testcase Name  : substr05.aql
- * Description    : Test substring2(string,position,position) built in function.
+ * Description    : Test substring(string,position,position) built in function.
  * Success        : Yes
  * Date           : 19th April 2012
  */
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
index 1b2cfe5..5c0fa4f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
@@ -1,14 +1,14 @@
 /*
  * Testcase Name  : substr05.aql
- * Description    : Test substring2(string,position,position) built in function.
+ * Description    : Test substring(string,position,position) built in function.
  * Success        : Yes
  * Date           : 19th April 2012
  */
 
-// To test substring2 function with string data stored in an internal dataset.
+// To test substring function with string data stored in an internal dataset.
 
 use dataverse test;
 
 for $a in dataset('testdst')
 order by $a.name
-return substring($a.name,4,string-length($a.name));
+return substring($a.name, 4, string-length($a.name) - 3);
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.1.ddl.aql
new file mode 100644
index 0000000..66d955c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.1.ddl.aql
@@ -0,0 +1,25 @@
+/*
+ * Test case name: insert_from_ext_ds_2
+ * Description: verify external data loading on temporal types
+ * Expected result: success
+ */
+ 
+drop dataverse timeTest if exists;
+create dataverse timeTest;
+use dataverse timeTest;
+
+create type timesType as open {
+    date: date,
+    time: time,
+    datetime: datetime,
+    duration: duration,
+    year-month-duration: year-month-duration,
+    day-time-duration: day-time-duration,
+	date-interval: interval,
+	time-interval: interval,
+	datetime-interval: interval
+}
+
+create dataset timeData(timesType)
+primary key date;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.2.update.aql
new file mode 100644
index 0000000..daff7c6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse timeTest;
+
+load dataset timeData using localfs
+(("path"="nc1://data/temporal/simpletemp_30.json"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.3.query.aql
new file mode 100644
index 0000000..28549c8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.3.query.aql
@@ -0,0 +1,5 @@
+use dataverse timeTest;
+
+for $r in dataset timeData
+order by $r.date
+return $r
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.1.ddl.aql
new file mode 100644
index 0000000..a6af75e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.1.ddl.aql
@@ -0,0 +1,44 @@
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+use dataverse TinySocial;
+
+create type TwitterUserType as open {
+        screen-name: string,
+        lang: string,
+        friends_count: int32,
+        statuses_count: int32,
+        name: string,
+        followers_count: int32
+}
+
+create type TweetMessageType as closed {
+        tweetid: string,
+        user: TwitterUserType,
+        sender-location: point?,
+        send-time: datetime,
+        referred-topics: {{ string }},
+        message-text: string
+}
+
+create type EmploymentType as open {
+        organization-name: string,      
+        start-date: date,
+        end-date: date?
+}
+
+create type FacebookUserType as closed {
+        id: int32,
+        alias: string,
+        name: string,
+        user-since: datetime,
+        friend-ids: {{ int32 }},
+        employment: [EmploymentType]
+}
+
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.10.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.10.query.aql
new file mode 100644
index 0000000..24da0fb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.10.query.aql
@@ -0,0 +1,15 @@
+/**
+* Query 4 - Theta Join
+*/
+
+use dataverse TinySocial;
+
+for $t in dataset TweetMessages
+order by $t.message-text
+return {
+"message": $t.message-text,
+"nearby-messages": for $t2 in dataset TweetMessages
+                        where spatial-distance($t.sender-location, $t2.sender-location) <= 1
+                        order by $t2.message-text
+                        return { "msgtxt":$t2.message-text}
+};
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.11.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.11.query.aql
new file mode 100644
index 0000000..5f428cd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.11.query.aql
@@ -0,0 +1,22 @@
+/**
+* Query 5 - Fuzzy Join 
+*/
+
+use dataverse TinySocial;
+
+set simfunction "edit-distance";
+set simthreshold "3";
+
+for $fbu in dataset FacebookUsers
+order by $fbu.id
+return {
+    "id": $fbu.id,
+    "name": $fbu.name,
+    "similar-users": for $t in dataset TweetMessages
+                        let $tu := $t.user
+                        where $tu.name ~= $fbu.name
+                        return {
+                        "twitter-screenname": $tu.screen-name,
+                        "twitter-name": $tu.name
+                        }
+};
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.12.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.12.query.aql
new file mode 100644
index 0000000..869b4ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.12.query.aql
@@ -0,0 +1,10 @@
+/**
+* Query 6 - Existential Quantification
+*/
+
+use dataverse TinySocial;
+
+for $fbu in dataset FacebookUsers
+where (some $e in $fbu.employment satisfies is-null($e.end-date)) 
+order by $fbu.id
+return $fbu;
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.13.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.13.query.aql
new file mode 100644
index 0000000..056cf9a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.13.query.aql
@@ -0,0 +1,11 @@
+/**
+* Query 7 - Universal Quantification
+*/
+
+use dataverse TinySocial;
+
+
+for $fbu in dataset FacebookUsers
+where (every $e in $fbu.employment satisfies not(is-null($e.end-date))) 
+order by $fbu.id
+return $fbu;
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.14.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.14.query.aql
new file mode 100644
index 0000000..5593669
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.14.query.aql
@@ -0,0 +1,7 @@
+/**
+* Query 8 - Simple Aggregation
+*/
+
+use dataverse TinySocial;
+
+count(for $fbu in dataset FacebookUsers return $fbu);
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.15.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.15.query.aql
new file mode 100644
index 0000000..2b45b00
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.15.query.aql
@@ -0,0 +1,13 @@
+/**
+* Query 9-A - Grouping and Aggregation
+*/
+
+use dataverse TinySocial;
+
+for $t in dataset TweetMessages
+group by $uid := $t.user.screen-name with $t
+order by $uid
+return {
+"user": $uid,
+"count": count($t)
+};
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.16.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.16.query.aql
new file mode 100644
index 0000000..22c1e50
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.16.query.aql
@@ -0,0 +1,14 @@
+/**
+* Query 9-B - (Hash-Based) Grouping and Aggregation
+*/
+
+use dataverse TinySocial;
+
+for $t in dataset TweetMessages
+/*+ hash*/
+group by $uid := $t.user.screen-name with $t
+order by $uid
+return {
+"user": $uid,
+"count": count($t)
+};
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.17.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.17.query.aql
new file mode 100644
index 0000000..c29c966
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.17.query.aql
@@ -0,0 +1,15 @@
+/**
+* Query 10 - Grouping and Limits
+*/
+
+use dataverse TinySocial;
+
+for $t in dataset TweetMessages
+group by $uid := $t.user.screen-name with $t
+let $c := count($t)
+order by  $c,$uid desc
+limit 3
+return {
+        "user": $uid,
+        "count": $c
+};
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.18.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.18.query.aql
new file mode 100644
index 0000000..2d69ba2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.18.query.aql
@@ -0,0 +1,19 @@
+/**
+* Query 11 - Left Outer Fuzzy Join
+*/
+
+use dataverse TinySocial;
+
+set simfunction "jaccard";
+set simthreshold "0.3";
+
+for $t in dataset TweetMessages
+order by $t.tweetid
+return {                     
+    "tweet": $t,               
+    "similar-tweets": for $t2 in dataset TweetMessages
+                        where  $t2.referred-topics ~= $t.referred-topics
+                        and $t2.tweetid != $t.tweetid
+                        order by $t2.tweetid
+                        return $t2.referred-topics
+};
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.19.update.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.19.update.aql
new file mode 100644
index 0000000..2a6fd83
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.19.update.aql
@@ -0,0 +1,23 @@
+/**
+* Inserting New Data
+*/
+
+use dataverse TinySocial;
+
+insert into dataset TweetMessages
+(
+   {"tweetid":"13",
+    "user":
+        {"screen-name":"NathanGiesen@211",
+         "lang":"en",
+         "friends_count":39345,
+         "statuses_count":479,
+         "name":"Nathan Giesen",
+         "followers_count":49420
+        },
+    "sender-location":point("47.44,80.65"),
+    "send-time":datetime("2008-04-26T10:10:35"),
+    "referred-topics":{{"tweeting"}},
+    "message-text":"tweety tweet, my fellow tweeters!"
+   }
+);
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.2.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.2.ddl.aql
new file mode 100644
index 0000000..1fcc52e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.2.ddl.aql
@@ -0,0 +1,19 @@
+use dataverse TinySocial;
+
+create dataset FacebookUsers(FacebookUserType)
+primary key id;
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id;
+
+create dataset TwitterUsers(TwitterUserType)
+primary key screen-name;
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid
+hints(cardinality=100); 
+
+create index fbUserSinceIdx on FacebookUsers(user-since);
+create index fbAuthorIdx on FacebookMessages(author-id) type btree;
+create index fbSenderLocIndex on FacebookMessages(sender-location) type rtree;
+create index fbMessageIdx on FacebookMessages(message) type keyword;
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.20.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.20.query.aql
new file mode 100644
index 0000000..ee7bb3f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.20.query.aql
@@ -0,0 +1,9 @@
+/**
+* Inserting New Data - Verification
+*/
+
+use dataverse TinySocial;
+
+for $t in dataset TweetMessages
+order by $t.tweetid
+return $t
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.21.update.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.21.update.aql
new file mode 100644
index 0000000..a8912cc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.21.update.aql
@@ -0,0 +1,7 @@
+/**
+* Deleting Existing Data
+*/
+
+use dataverse TinySocial;
+
+delete $tm from dataset TweetMessages where $tm.tweetid = "13";
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.22.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.22.query.aql
new file mode 100644
index 0000000..395b7c1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.22.query.aql
@@ -0,0 +1,11 @@
+/**
+* Deleting Existing Data - Verification
+*/
+
+use dataverse TinySocial;
+
+count(
+for $t in dataset TweetMessages
+where $t.tweetid = "13"
+return $t
+)
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.3.update.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.3.update.aql
new file mode 100644
index 0000000..92db8a1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.3.update.aql
@@ -0,0 +1,13 @@
+use dataverse TinySocial;
+
+load dataset FacebookUsers using localfs
+(("path"="nc1://data/tinysocial/fbu.adm"),("format"="adm"));
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/tinysocial/fbm.adm"),("format"="adm"));
+
+load dataset TwitterUsers using localfs
+(("path"="nc1://data/tinysocial/twu.adm"),("format"="adm"));
+
+load dataset TweetMessages using localfs
+(("path"="nc1://data/tinysocial/twm.adm"),("format"="adm"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.4.query.aql
new file mode 100644
index 0000000..0bf0d89
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.4.query.aql
@@ -0,0 +1,9 @@
+/**
+* Query 0-A - Exact-Match Lookup
+*/
+
+use dataverse TinySocial;
+
+for $user in dataset FacebookUsers
+where $user.id = 8
+return $user;
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.5.query.aql
new file mode 100644
index 0000000..a7b40de
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.5.query.aql
@@ -0,0 +1,10 @@
+/**
+* Query 0-B - Range Scan
+*/
+
+use dataverse TinySocial;
+
+for $user in dataset FacebookUsers
+where $user.id >= 2 and $user.id <= 4
+order by $user.id
+return $user;
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.6.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.6.query.aql
new file mode 100644
index 0000000..40234f6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.6.query.aql
@@ -0,0 +1,11 @@
+/**
+* Query 1 - Other Query Filters 
+*/
+
+use dataverse TinySocial;
+
+for $user in dataset FacebookUsers
+where $user.user-since >= datetime('2010-07-22T00:00:00')
+  and $user.user-since <= datetime('2012-07-29T23:59:59')
+order by $user.id
+return $user;
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.7.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.7.query.aql
new file mode 100644
index 0000000..98112ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.7.query.aql
@@ -0,0 +1,14 @@
+/**
+* Query 2-A - Equijoin
+*/
+
+use dataverse TinySocial;
+
+for $user in dataset FacebookUsers
+for $message in dataset FacebookMessages
+where $message.author-id = $user.id 
+order by $user.name,$message.message-id
+return {
+"uname": $user.name,
+"message": $message.message
+};
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.8.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.8.query.aql
new file mode 100644
index 0000000..e274571
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.8.query.aql
@@ -0,0 +1,14 @@
+/**
+* Query 2-B - Index join
+*/
+
+use dataverse TinySocial;
+
+for $user in dataset FacebookUsers
+for $message in dataset FacebookMessages
+where $message.author-id /*+ indexnl */  = $user.id
+order by $user.name,$message.message-id
+return {
+"uname": $user.name,
+"message": $message.message
+};
diff --git a/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.9.query.aql b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.9.query.aql
new file mode 100644
index 0000000..c4e70c6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.9.query.aql
@@ -0,0 +1,15 @@
+/**
+* Query 3 - Nested Outer Join
+*/
+
+use dataverse TinySocial;
+
+for $user in dataset FacebookUsers
+order by $user.name
+return {
+"uname": $user.name,
+"messages": for $message in dataset FacebookMessages
+                where $message.author-id = $user.id
+                order by $message.message-id
+                return $message.message
+};
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.3.query.aql
index f61ceb5..9f1080d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.3.query.aql
@@ -17,7 +17,7 @@
 
 for $s in dataset('Supplier')
 for $r in revenue()
-	where $s.s_suppkey = $r.supplier_no and $r.total_revenue=$m
+	where $s.s_suppkey = $r.supplier_no and $r.total_revenue<$m+0.000000001 and $r.total_revenue>$m-0.000000001
 return {
 "s_suppkey": $s.s_suppkey, 
 "s_name": $s.s_name, 
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue172/query-issue172.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue172/query-issue172.1.ddl.aql
new file mode 100644
index 0000000..e1fab96
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue172/query-issue172.1.ddl.aql
@@ -0,0 +1,7 @@
+/*
+ * Description  : This test case is to verify the fix for issue172
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=172
+ * Expected Res : Success
+ * Date         : 19th May 2013
+ */
+ 
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue172/query-issue172.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue172/query-issue172.2.update.aql
new file mode 100644
index 0000000..b08fc74
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue172/query-issue172.2.update.aql
@@ -0,0 +1,7 @@
+/*
+ * Description  : This test case is to verify the fix for issue172
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=172
+ * Expected Res : Success
+ * Date         : 19th May 2013
+ */
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue172/query-issue172.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue172/query-issue172.3.query.aql
new file mode 100644
index 0000000..b695ccb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue172/query-issue172.3.query.aql
@@ -0,0 +1,9 @@
+/*
+ * Description  : This test case is to verify the fix for issue172
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=172
+ * Expected Res : Success
+ * Date         : 19th May 2013
+ */
+
+let $a := string-length(string-concat(["this is a ","test string"]))
+return $a
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue244/query-issue244.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue244/query-issue244.1.ddl.aql
new file mode 100644
index 0000000..6f7b5e0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue244/query-issue244.1.ddl.aql
@@ -0,0 +1,25 @@
+/*
+ * Description  : This test case is to verify the fix for issue244
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=244
+ * Expected Res : Success
+ * Date         : 4th June 2013
+ */
+
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type TypeA as open {
+    id : int32,
+    name : string
+}
+
+create dataset t1(TypeA) primary key id;
+
+create dataset t2(TypeA) primary key id;
+
+create function f1(){
+for $m in dataset('t1') return $m
+};
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue244/query-issue244.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue244/query-issue244.2.update.aql
new file mode 100644
index 0000000..a7b9500
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue244/query-issue244.2.update.aql
@@ -0,0 +1,16 @@
+/*
+ * Description  : This test case is to verify the fix for issue244
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=244
+ * Expected Res : Success
+ * Date         : 4th June 2013
+ */
+
+use dataverse test;
+ 
+insert into dataset t1({"id":21,"name":"John"});
+insert into dataset t1({"id":34,"name":"Bill"});
+insert into dataset t1({"id":41,"name":"Joy"});
+insert into dataset t1({"id":16,"name":"Sam"});
+insert into dataset t1({"id":67,"name":"Ravi"});
+
+insert into dataset t2(f1());
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue244/query-issue244.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue244/query-issue244.3.query.aql
new file mode 100644
index 0000000..68e4679
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue244/query-issue244.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue244
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=244
+ * Expected Res : Success
+ * Date         : 4th June 2013
+ */
+ 
+use dataverse test;
+ 
+for $l in dataset('t2')
+order by $l.id
+return $l;
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.1.ddl.aql
new file mode 100644
index 0000000..93b9131
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.1.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Description  : This test case is to verify the fix for issue 489
+ 	          https://code.google.com/p/asterixdb/issues/detail?id=489
+ * Expected Res : Success
+ * Date         : 31st May 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create function f1()
+{
+"function with no input"
+}
+
+create function f1($a)
+{
+"function with input"
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.2.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.2.query.aql
new file mode 100644
index 0000000..4dcf6e6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.2.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue 489
+ 	          https://code.google.com/p/asterixdb/issues/detail?id=489
+ * Expected Res : Success
+ * Date         : 31st May 2013
+ */
+
+count(
+for $x in dataset Metadata.Function
+where $x.DataverseName='test'
+return $x
+)
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.3.ddl.aql
new file mode 100644
index 0000000..867ae7a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.3.ddl.aql
@@ -0,0 +1,10 @@
+/*
+ * Description  : This test case is to verify the fix for issue 489
+ 	          https://code.google.com/p/asterixdb/issues/detail?id=489
+ * Expected Res : Success
+ * Date         : 31st May 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.4.query.aql
new file mode 100644
index 0000000..4dcf6e6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue489/query-issue489.4.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue 489
+ 	          https://code.google.com/p/asterixdb/issues/detail?id=489
+ * Expected Res : Success
+ * Date         : 31st May 2013
+ */
+
+count(
+for $x in dataset Metadata.Function
+where $x.DataverseName='test'
+return $x
+)
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf29/udf29.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf29/udf29.1.ddl.aql
index edede00..f9f1e25 100644
--- a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf29/udf29.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf29/udf29.1.ddl.aql
@@ -8,9 +8,3 @@
 drop dataverse test if exists;
 create dataverse test;
 
-use dataverse test;
-
-declare function test.f1(){
-100
-};
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf29/udf29.2.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf29/udf29.2.query.aql
index 2f9e763..5755963 100644
--- a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf29/udf29.2.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf29/udf29.2.query.aql
@@ -5,5 +5,10 @@
  */
 
 use dataverse test;
+
+declare function f1(){
+100
+};
+
 let $x:=f1()
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/opentype/opentype.1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/opentype/opentype.1.adm
new file mode 100644
index 0000000..ae0c4cb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/opentype/opentype.1.adm
@@ -0,0 +1,12 @@
+{ "tweet": { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, "similar-tweets": [ {{ "t-mobile", "shortcut-menu" }} ] }
+{ "tweet": { "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, "similar-tweets": [ {{ "verizon", "shortcut-menu" }}, {{ "iphone", "voice-clarity" }}, {{ "verizon", "voicemail-service" }} ] }
+{ "tweet": { "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, "similar-tweets": [ {{ "iphone", "voice-clarity" }}, {{ "samsung", "platform" }} ] }
+{ "tweet": { "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, "similar-tweets": [ {{ "sprint", "voice-command" }}, {{ "samsung", "platform" }} ] }
+{ "tweet": { "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "t-mobile", "shortcut-menu" }}, {{ "verizon", "voicemail-service" }} ] }
+{ "tweet": { "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
+{ "tweet": { "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, "similar-tweets": [ {{ "samsung", "voice-command" }} ] }
+{ "tweet": { "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
+{ "tweet": { "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "iphone", "platform" }} ] }
+{ "tweet": { "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, "similar-tweets": [ {{ "iphone", "platform" }}, {{ "samsung", "voice-command" }} ] }
+{ "tweet": { "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "t-mobile", "customization" }}, {{ "verizon", "shortcut-menu" }} ] }
+{ "tweet": { "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "verizon", "shortcut-menu" }} ] }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/hdfs/issue_245_hdfs/issue_245_hdfs.1.adm b/asterix-app/src/test/resources/runtimets/results/hdfs/issue_245_hdfs/issue_245_hdfs.1.adm
index 8af2f5f..59425b1 100644
--- a/asterix-app/src/test/resources/runtimets/results/hdfs/issue_245_hdfs/issue_245_hdfs.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/hdfs/issue_245_hdfs/issue_245_hdfs.1.adm
@@ -1,4 +1,4 @@
+{ "line": "ASTERIX is taking an open stance on data formats and addressing research issues including highly scalable data storage and indexing, semi-structured query processing on very large clusters, and merging parallel database techniques with todays data-intensive computing techniques to support performant yet declarative solutions to the problem of analyzing semi-structured information" }
+{ "line": "ASTERIX targets a wide range of semi-structured information, ranging from data use cases where information is well-tagged and highly regular to content use cases where data is irregular and much of each datum is textual" }
 { "line": "The ASTERIX project is developing new technologies for ingesting, storing, managing, indexing, querying, analyzing, and subscribing to vast quantities of semi-structured information" }
 { "line": "The project is combining ideas from three distinct areas semi-structured data, parallel databases, and data-intensive computing  to create a next-generation, open source software platform that scales by running on large, shared-nothing commodity computing clusters" }
-{ "line": "ASTERIX targets a wide range of semi-structured information, ranging from data use cases where information is well-tagged and highly regular to content use cases where data is irregular and much of each datum is textual" }
-{ "line": "ASTERIX is taking an open stance on data formats and addressing research issues including highly scalable data storage and indexing, semi-structured query processing on very large clusters, and merging parallel database techniques with todays data-intensive computing techniques to support performant yet declarative solutions to the problem of analyzing semi-structured information" }
diff --git a/asterix-app/src/test/resources/runtimets/results/misc/flushtest/flushtest.1.adm b/asterix-app/src/test/resources/runtimets/results/misc/flushtest/flushtest.1.adm
new file mode 100644
index 0000000..a880bd1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/misc/flushtest/flushtest.1.adm
@@ -0,0 +1,1000 @@
+{ "id": 9005038, "id-copy": 9005038, "alias": "Anabel", "name": "AnabelWheeler", "user-since": datetime("2006-12-12T13:40:23.000Z"), "user-since-copy": datetime("2006-12-12T13:40:23.000Z"), "friend-ids": {{ 18713256, 35193719, 42245821, 37249622, 12210708, 15557948, 467039, 43997520, 45171035, 43682410, 47884198, 43102086, 39620955, 36438278, 42976932, 11158113, 21543594, 9861181, 36944403, 47928849, 29593861, 37897057, 42360015, 27956902 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2000-03-21"), "end-date": null } ] }
+{ "id": 9029377, "id-copy": 9029377, "alias": "Boyce", "name": "BoyceAnderson", "user-since": datetime("2010-12-18T14:17:12.000Z"), "user-since-copy": datetime("2010-12-18T14:17:12.000Z"), "friend-ids": {{ 19260027, 21449100, 35898407, 34501982 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2007-06-25"), "end-date": null } ] }
+{ "id": 9041443, "id-copy": 9041443, "alias": "Maria", "name": "MariaWard", "user-since": datetime("2006-12-25T01:24:40.000Z"), "user-since-copy": datetime("2006-12-25T01:24:40.000Z"), "friend-ids": {{ 10660010, 19103672, 11300656, 44383404, 36523093, 11434370, 34405687, 30889551, 4843181, 22025114, 26395363, 8607483, 25294309 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2005-06-13"), "end-date": null } ] }
+{ "id": 9045535, "id-copy": 9045535, "alias": "Ebenezer", "name": "EbenezerPery", "user-since": datetime("2008-06-05T17:48:45.000Z"), "user-since-copy": datetime("2008-06-05T17:48:45.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2012-04-07"), "end-date": date("2012-06-10") } ] }
+{ "id": 9046852, "id-copy": 9046852, "alias": "Mauro", "name": "MauroChase", "user-since": datetime("2011-04-18T20:18:58.000Z"), "user-since-copy": datetime("2011-04-18T20:18:58.000Z"), "friend-ids": {{ 28268506, 13880377, 18637778, 27129860, 47146036, 23136396, 34534506, 23274864, 38781071, 9644011, 34754620, 45178277, 33832472, 31871984, 47201051, 42153557, 12418584, 37615805, 35474951, 29273401, 4845352, 18687033 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2012-05-14"), "end-date": date("2012-06-25") } ] }
+{ "id": 9050164, "id-copy": 9050164, "alias": "Haydee", "name": "HaydeeCook", "user-since": datetime("2005-08-28T12:13:59.000Z"), "user-since-copy": datetime("2005-08-28T12:13:59.000Z"), "friend-ids": {{ 26484166, 27686644, 42277018, 5893537, 34617524, 12158738, 41566344, 30653024, 23636324, 24072660, 1784294, 38620941, 40846838, 30303402, 27004887, 35907658, 42893556, 10118575, 47861482 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2007-11-23"), "end-date": null } ] }
+{ "id": 9067279, "id-copy": 9067279, "alias": "Jeanine", "name": "JeanineEmrick", "user-since": datetime("2011-06-25T09:43:07.000Z"), "user-since-copy": datetime("2011-06-25T09:43:07.000Z"), "friend-ids": {{ 12884712, 45104617, 41134568, 15844605, 645264, 33182092, 16884335, 46281324, 3977219, 5682848, 441588, 26025738, 3165091, 21821928, 23073877 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2012-01-02"), "end-date": null } ] }
+{ "id": 9074290, "id-copy": 9074290, "alias": "Riley", "name": "RileyBode", "user-since": datetime("2010-11-20T01:12:36.000Z"), "user-since-copy": datetime("2010-11-20T01:12:36.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2004-10-12"), "end-date": null } ] }
+{ "id": 9082201, "id-copy": 9082201, "alias": "Alberic", "name": "AlbericCrawford", "user-since": datetime("2005-02-11T07:41:05.000Z"), "user-since-copy": datetime("2005-02-11T07:41:05.000Z"), "friend-ids": {{ 26925567, 6108069, 30484049, 4903722, 4579631, 21166966, 3892344, 6259030, 32887933, 7183018, 46041497, 23448710, 47887528, 3679587, 7140571, 47671072, 4554470, 23481403, 16738975, 4885244 }}, "employment": [ { "organization-name": "Voltbam", "start-date": date("2006-10-10"), "end-date": null } ] }
+{ "id": 9098314, "id-copy": 9098314, "alias": "Terrance", "name": "TerranceWilkerson", "user-since": datetime("2010-07-01T06:01:32.000Z"), "user-since-copy": datetime("2010-07-01T06:01:32.000Z"), "friend-ids": {{ 32477103, 38306013, 36022406, 25594192, 10966661, 28537611, 5444323, 16012053, 43228208, 30344050, 22600011, 42820310, 37103995, 6359985 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2012-01-24"), "end-date": null } ] }
+{ "id": 9133714, "id-copy": 9133714, "alias": "Wil", "name": "WilDale", "user-since": datetime("2009-12-04T18:40:04.000Z"), "user-since-copy": datetime("2009-12-04T18:40:04.000Z"), "friend-ids": {{ 40400811, 26528322 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2005-10-08"), "end-date": date("2007-03-23") } ] }
+{ "id": 9139057, "id-copy": 9139057, "alias": "Esther", "name": "EstherUllman", "user-since": datetime("2010-01-05T19:25:44.000Z"), "user-since-copy": datetime("2010-01-05T19:25:44.000Z"), "friend-ids": {{ 25401186, 25915246, 33727208, 17431690, 24541706, 19998503, 42399029, 30405906, 20023918, 9788811, 32513474, 14919034, 10073867, 9309154, 1423378, 37386209, 16346279, 45167618, 34716280, 29023237, 20639001, 332097, 28344544 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2003-09-05"), "end-date": date("2009-10-17") } ] }
+{ "id": 9142198, "id-copy": 9142198, "alias": "Sherry", "name": "SherryFea", "user-since": datetime("2011-03-28T23:09:22.000Z"), "user-since-copy": datetime("2011-03-28T23:09:22.000Z"), "friend-ids": {{ 6835080, 34471872, 30942941, 34858577, 5996593, 47293442, 43097072, 44809621, 33969893, 26410931, 6628186, 29944391, 35957320, 20326929, 40284077, 11681583, 43878314, 40265961, 16871274, 28406169, 1349311 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2004-07-28"), "end-date": null } ] }
+{ "id": 9185848, "id-copy": 9185848, "alias": "Brendon", "name": "BrendonJelliman", "user-since": datetime("2008-10-13T17:36:00.000Z"), "user-since-copy": datetime("2008-10-13T17:36:00.000Z"), "friend-ids": {{ 12675636, 6787931, 19218962, 12655930 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2008-06-09"), "end-date": date("2009-10-16") } ] }
+{ "id": 9203731, "id-copy": 9203731, "alias": "Phoebe", "name": "PhoebeCoates", "user-since": datetime("2008-04-27T01:42:34.000Z"), "user-since-copy": datetime("2008-04-27T01:42:34.000Z"), "friend-ids": {{ 25611465, 519838, 22814080, 46015954, 7805914, 12757618, 36785422, 25727822, 32042543 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2003-11-11"), "end-date": date("2005-08-19") } ] }
+{ "id": 9211711, "id-copy": 9211711, "alias": "Seraphina", "name": "SeraphinaFlanders", "user-since": datetime("2009-05-19T18:39:15.000Z"), "user-since-copy": datetime("2009-05-19T18:39:15.000Z"), "friend-ids": {{ 34432294, 10796959, 46386746, 32318131, 10393677, 12832313, 34490791, 6187782, 46595448, 30591963, 35530646, 22485004, 18950892, 19762388, 19181134, 13928403, 22513246, 24969298 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2008-05-14"), "end-date": date("2009-06-17") } ] }
+{ "id": 9212815, "id-copy": 9212815, "alias": "Erica", "name": "EricaBraun", "user-since": datetime("2009-01-11T07:32:03.000Z"), "user-since-copy": datetime("2009-01-11T07:32:03.000Z"), "friend-ids": {{ 1314906, 6581233, 35117578, 11133528, 19606776, 37833518, 40040803, 44107209, 38804989, 35779440, 41138709 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2008-04-03"), "end-date": null } ] }
+{ "id": 9221836, "id-copy": 9221836, "alias": "Claud", "name": "ClaudPratt", "user-since": datetime("2008-01-01T04:10:02.000Z"), "user-since-copy": datetime("2008-01-01T04:10:02.000Z"), "friend-ids": {{ 35586361, 40548794, 7169299, 24675214, 21079165, 37323851, 16881366, 24433012, 38047831, 34495409, 33711705, 8957126, 38345318 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2002-12-05"), "end-date": null } ] }
+{ "id": 9223375, "id-copy": 9223375, "alias": "Anne", "name": "AnneMoore", "user-since": datetime("2010-07-16T22:06:20.000Z"), "user-since-copy": datetime("2010-07-16T22:06:20.000Z"), "friend-ids": {{ 45553359, 40589681, 9461257, 39253068, 14447226, 37656564, 37047377, 34855985 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2011-04-25"), "end-date": null } ] }
+{ "id": 9232504, "id-copy": 9232504, "alias": "Lesley", "name": "LesleyHujsak", "user-since": datetime("2008-07-07T13:30:22.000Z"), "user-since-copy": datetime("2008-07-07T13:30:22.000Z"), "friend-ids": {{ 42058063, 24501683, 26865036, 180621 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2011-01-04"), "end-date": date("2011-02-07") } ] }
+{ "id": 9234529, "id-copy": 9234529, "alias": "Xavior", "name": "XaviorBarnes", "user-since": datetime("2010-08-26T12:06:44.000Z"), "user-since-copy": datetime("2010-08-26T12:06:44.000Z"), "friend-ids": {{ 19552290, 24018104, 43285028, 33954718, 18084047, 18675363, 17369450, 36533551, 46779811, 46943171, 17609996, 14171942, 10468121, 33831228, 9905114, 11839935, 41387228 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2007-12-24"), "end-date": null } ] }
+{ "id": 9262768, "id-copy": 9262768, "alias": "Graham", "name": "GrahamHunt", "user-since": datetime("2009-03-19T13:15:02.000Z"), "user-since-copy": datetime("2009-03-19T13:15:02.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2012-04-23"), "end-date": date("2012-04-15") } ] }
+{ "id": 9265747, "id-copy": 9265747, "alias": "Nicolas", "name": "NicolasPirl", "user-since": datetime("2011-11-07T13:52:49.000Z"), "user-since-copy": datetime("2011-11-07T13:52:49.000Z"), "friend-ids": {{ 5832017, 30839617, 27328653, 9766355, 35973149, 21029594, 18840511, 43035135, 44902336, 11576374, 21756219, 23374243, 42201568, 12860309 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2003-11-10"), "end-date": date("2010-03-27") } ] }
+{ "id": 9286279, "id-copy": 9286279, "alias": "Barnaby", "name": "BarnabyAckerley", "user-since": datetime("2006-09-15T01:56:34.000Z"), "user-since-copy": datetime("2006-09-15T01:56:34.000Z"), "friend-ids": {{ 21236050, 22647474, 18898492, 22530993, 4332450, 38947319, 25882415, 47187086, 5810354, 18396369, 44918707, 9732196, 14821426, 148735 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2011-02-10"), "end-date": null } ] }
+{ "id": 9288154, "id-copy": 9288154, "alias": "Lauren", "name": "LaurenGraff", "user-since": datetime("2005-12-28T07:21:17.000Z"), "user-since-copy": datetime("2005-12-28T07:21:17.000Z"), "friend-ids": {{ 38658043, 4029859, 43671010, 20184796, 23429992, 3744331, 39377881, 1336305, 33712064, 36443 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2009-04-06"), "end-date": null } ] }
+{ "id": 9313492, "id-copy": 9313492, "alias": "Tera", "name": "TeraWolfe", "user-since": datetime("2010-12-20T12:47:25.000Z"), "user-since-copy": datetime("2010-12-20T12:47:25.000Z"), "friend-ids": {{ 45424983, 18345704, 14849759, 31638064, 38670515, 48015953, 36114769 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2001-04-26"), "end-date": date("2004-12-06") } ] }
+{ "id": 9331075, "id-copy": 9331075, "alias": "Monday", "name": "MondayWarrick", "user-since": datetime("2012-01-13T06:13:30.000Z"), "user-since-copy": datetime("2012-01-13T06:13:30.000Z"), "friend-ids": {{ 27699724, 39094128, 11014820, 44605243, 20177679, 37579779, 35875781, 13713739, 8882475, 37427927, 28595578, 3788567, 31200715, 40590973, 7630783, 36856789, 22013865 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2000-04-08"), "end-date": null } ] }
+{ "id": 9372871, "id-copy": 9372871, "alias": "Emerson", "name": "EmersonSell", "user-since": datetime("2010-01-25T11:12:56.000Z"), "user-since-copy": datetime("2010-01-25T11:12:56.000Z"), "friend-ids": {{ 13800934, 24493814 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2004-02-14"), "end-date": date("2005-11-07") } ] }
+{ "id": 9373726, "id-copy": 9373726, "alias": "Joe", "name": "JoeRoche", "user-since": datetime("2005-07-09T16:42:53.000Z"), "user-since-copy": datetime("2005-07-09T16:42:53.000Z"), "friend-ids": {{ 16433644, 5532847, 743901, 2134179, 43053028, 36961668, 9731766, 45686582, 17084459, 27026683, 1687547, 6582422, 38798685, 9871595, 2677099, 42280963, 32191501, 4347234 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2009-09-16"), "end-date": null } ] }
+{ "id": 9403096, "id-copy": 9403096, "alias": "Clarita", "name": "ClaritaRitter", "user-since": datetime("2007-11-18T14:11:04.000Z"), "user-since-copy": datetime("2007-11-18T14:11:04.000Z"), "friend-ids": {{ 11967380, 17558867 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2011-01-28"), "end-date": date("2011-05-05") } ] }
+{ "id": 9426244, "id-copy": 9426244, "alias": "Lamar", "name": "LamarMaugham", "user-since": datetime("2005-03-08T17:00:15.000Z"), "user-since-copy": datetime("2005-03-08T17:00:15.000Z"), "friend-ids": {{ 36168436, 20740167, 21922111, 32892152, 34608833, 28621520, 40818313, 23842558, 41275216, 36331147, 40737858, 45983619, 14033949, 23132425, 33634408 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2011-12-24"), "end-date": null } ] }
+{ "id": 9440452, "id-copy": 9440452, "alias": "Maria", "name": "MariaField", "user-since": datetime("2010-04-06T15:15:24.000Z"), "user-since-copy": datetime("2010-04-06T15:15:24.000Z"), "friend-ids": {{ 35137543, 24166956, 45255343, 10050289, 27769291, 40368984, 38146662, 43123957, 10442976, 46931482, 447566, 14148069, 39035817, 32169234, 35607837, 8648749, 3741547, 31840808, 3029722, 40917859 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2012-04-27"), "end-date": date("2012-05-11") } ] }
+{ "id": 9442978, "id-copy": 9442978, "alias": "Osborne", "name": "OsborneHiles", "user-since": datetime("2012-07-28T10:59:39.000Z"), "user-since-copy": datetime("2012-07-28T10:59:39.000Z"), "friend-ids": {{ 40833026, 39533118, 6206868, 27383373, 3010465, 14776443, 43239645, 21956253, 4112089, 27667721, 34336067, 38377619, 32701403, 20907262, 32732275, 30488150, 12349697, 47468946, 20956164, 16141416 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2011-08-21"), "end-date": null } ] }
+{ "id": 9477994, "id-copy": 9477994, "alias": "Cory", "name": "CoryKeener", "user-since": datetime("2012-02-27T22:03:31.000Z"), "user-since-copy": datetime("2012-02-27T22:03:31.000Z"), "friend-ids": {{ 22204843, 35394804, 22795967, 16575437, 31764908, 27359073, 50023, 26383393, 36534917, 23478654, 31022293, 43803666, 24764841, 19469389, 6401330, 10543085, 5159571 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2012-02-09"), "end-date": date("2012-02-19") } ] }
+{ "id": 9478720, "id-copy": 9478720, "alias": "Angelia", "name": "AngeliaKettlewell", "user-since": datetime("2005-05-27T06:29:30.000Z"), "user-since-copy": datetime("2005-05-27T06:29:30.000Z"), "friend-ids": {{ 42556433, 20033025, 38112512, 19420757, 31822717, 7116081, 39544900, 19203395, 46787205, 32303456, 4509345, 45558040, 42616291, 6929369, 9272653, 37459048, 37113569, 38942369, 47741031, 46761451, 14163845 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2012-03-28"), "end-date": date("2012-03-04") } ] }
+{ "id": 9502096, "id-copy": 9502096, "alias": "Hebe", "name": "HebeEndsley", "user-since": datetime("2012-08-08T18:55:28.000Z"), "user-since-copy": datetime("2012-08-08T18:55:28.000Z"), "friend-ids": {{ 34917916, 5530270, 12994124, 25113086, 28819142, 44228082 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2007-04-11"), "end-date": null } ] }
+{ "id": 9503443, "id-copy": 9503443, "alias": "Ebenezer", "name": "EbenezerFulton", "user-since": datetime("2012-07-03T20:14:05.000Z"), "user-since-copy": datetime("2012-07-03T20:14:05.000Z"), "friend-ids": {{ 11155403, 7932344, 24822329, 19823943, 37496284 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2011-08-22"), "end-date": null } ] }
+{ "id": 9516652, "id-copy": 9516652, "alias": "Emmanuel", "name": "EmmanuelStrickland", "user-since": datetime("2006-01-14T03:08:13.000Z"), "user-since-copy": datetime("2006-01-14T03:08:13.000Z"), "friend-ids": {{ 21213113, 8011145, 9382308, 14949454, 114459, 30046906, 40091327, 22275481, 14642211, 5602065, 15265189, 22736575, 12746303, 46033445, 17273286, 39395247, 6653955, 14664612, 35055957 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2011-10-15"), "end-date": null } ] }
+{ "id": 9522265, "id-copy": 9522265, "alias": "Brendon", "name": "BrendonLing", "user-since": datetime("2012-08-11T12:01:34.000Z"), "user-since-copy": datetime("2012-08-11T12:01:34.000Z"), "friend-ids": {{ 32770998, 43037450, 13481444, 36411834, 21704194 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2012-08-30"), "end-date": null } ] }
+{ "id": 9525361, "id-copy": 9525361, "alias": "Leonardo", "name": "LeonardoSurrency", "user-since": datetime("2008-12-21T10:09:26.000Z"), "user-since-copy": datetime("2008-12-21T10:09:26.000Z"), "friend-ids": {{ 12471014, 47714763, 18071069, 32545366, 46041462, 35261185, 20826834, 29002678, 47207065, 7370034, 38283272, 47090645, 33425043, 16014552, 15633873, 24101778, 26168621, 21955493, 17856723, 18158610 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2011-12-06"), "end-date": date("2011-04-04") } ] }
+{ "id": 9556570, "id-copy": 9556570, "alias": "Kassandra", "name": "KassandraKern", "user-since": datetime("2010-12-03T15:29:12.000Z"), "user-since-copy": datetime("2010-12-03T15:29:12.000Z"), "friend-ids": {{ 35944118, 3024691, 43927521, 44121317, 29834404, 18626717, 47095811, 38438153, 30557309, 37143411, 41634172, 23338449, 30455300, 12009022, 26366377, 36381324, 25084236, 36521163, 20063914, 11419154, 40243010, 9336807, 3544397, 20455720 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2005-02-12"), "end-date": null } ] }
+{ "id": 9568750, "id-copy": 9568750, "alias": "Daley", "name": "DaleyHarshman", "user-since": datetime("2012-01-17T10:38:07.000Z"), "user-since-copy": datetime("2012-01-17T10:38:07.000Z"), "friend-ids": {{ 18932212, 37118057, 37586464, 12686041, 21083532, 27598912 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2007-11-07"), "end-date": null } ] }
+{ "id": 9574393, "id-copy": 9574393, "alias": "Ghislaine", "name": "GhislaineTaylor", "user-since": datetime("2005-01-23T07:49:26.000Z"), "user-since-copy": datetime("2005-01-23T07:49:26.000Z"), "friend-ids": {{ 23799181, 25411427, 3758740, 47542325, 41070945, 45261892, 23309481 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2003-04-15"), "end-date": null } ] }
+{ "id": 9577867, "id-copy": 9577867, "alias": "Lavette", "name": "LavetteSnyder", "user-since": datetime("2007-02-22T10:01:04.000Z"), "user-since-copy": datetime("2007-02-22T10:01:04.000Z"), "friend-ids": {{ 25749553, 31379974, 15118772, 38725424, 26760226, 8908746, 20299291, 20288328, 19659485, 22400738, 477700, 20253845, 12753420, 46016251, 29518581, 21898853, 19015599, 3455762, 19350275, 2630122 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2011-04-22"), "end-date": null } ] }
+{ "id": 9596080, "id-copy": 9596080, "alias": "Yolonda", "name": "YolondaUlery", "user-since": datetime("2012-03-02T19:57:32.000Z"), "user-since-copy": datetime("2012-03-02T19:57:32.000Z"), "friend-ids": {{ 22382589, 22012001, 13142890, 44320162, 10358341, 14975, 43101433, 10324321, 14791134, 25984312, 11075173, 44140537, 40528755, 27384004, 40022140, 10650900, 37789740, 6928495, 22130557, 47679224, 40973393, 37190617, 35395183 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2012-03-03"), "end-date": null } ] }
+{ "id": 9629395, "id-copy": 9629395, "alias": "Julius", "name": "JuliusWire", "user-since": datetime("2008-03-22T13:36:24.000Z"), "user-since-copy": datetime("2008-03-22T13:36:24.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2006-11-19"), "end-date": null } ] }
+{ "id": 9629923, "id-copy": 9629923, "alias": "Adria", "name": "AdriaBoyer", "user-since": datetime("2005-08-12T16:31:38.000Z"), "user-since-copy": datetime("2005-08-12T16:31:38.000Z"), "friend-ids": {{ 43812176, 1271309, 1412045, 18793840, 40264072, 41525831, 25536841, 46110606, 40440782, 37228709, 37745315, 19025404, 13458371, 32475836, 30506186, 6860193, 44650222, 5924034 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2012-03-08"), "end-date": null } ] }
+{ "id": 9635563, "id-copy": 9635563, "alias": "Tamsen", "name": "TamsenCowart", "user-since": datetime("2010-10-07T05:11:20.000Z"), "user-since-copy": datetime("2010-10-07T05:11:20.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2007-01-07"), "end-date": null } ] }
+{ "id": 9638626, "id-copy": 9638626, "alias": "Hisako", "name": "HisakoEisaman", "user-since": datetime("2008-05-26T23:34:43.000Z"), "user-since-copy": datetime("2008-05-26T23:34:43.000Z"), "friend-ids": {{ 17773563, 18434504, 1082020, 40557107, 43294701, 1982610, 8259201, 47490886, 20044705, 35882471, 7297053, 17276976, 38660830, 36435103, 29511457, 3474864, 17100964, 23978369, 6260698, 17616437, 1617227, 18325960, 42613056 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2009-07-12"), "end-date": null } ] }
+{ "id": 9664990, "id-copy": 9664990, "alias": "Travis", "name": "TravisJube", "user-since": datetime("2010-02-12T13:42:04.000Z"), "user-since-copy": datetime("2010-02-12T13:42:04.000Z"), "friend-ids": {{ 22627931, 5992593, 8208547, 37326819, 14939087, 18366709, 29043862, 45062025, 21360937, 19730114, 26779317, 46856921, 28406774, 40580511, 8062361, 2179206, 47765870, 14039643, 28857662, 42600706 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2007-01-14"), "end-date": null } ] }
+{ "id": 9680644, "id-copy": 9680644, "alias": "Mirtha", "name": "MirthaRahl", "user-since": datetime("2008-02-09T04:05:03.000Z"), "user-since-copy": datetime("2008-02-09T04:05:03.000Z"), "friend-ids": {{ 25328638, 9009324, 16627989, 46602908, 32685062, 10538437, 22403363, 4205292, 27910567, 28430833, 8519372, 39774027, 12120028, 1211979 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2006-12-19"), "end-date": null } ] }
+{ "id": 9682723, "id-copy": 9682723, "alias": "Rick", "name": "RickEisaman", "user-since": datetime("2011-01-04T04:42:13.000Z"), "user-since-copy": datetime("2011-01-04T04:42:13.000Z"), "friend-ids": {{ 843458, 40779817, 24515616, 9016765, 37332064, 2164822, 45832315, 27168757, 43771964, 46638388, 43667809 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2004-08-13"), "end-date": date("2011-04-11") } ] }
+{ "id": 9709663, "id-copy": 9709663, "alias": "Trevor", "name": "TrevorSell", "user-since": datetime("2008-08-28T18:18:54.000Z"), "user-since-copy": datetime("2008-08-28T18:18:54.000Z"), "friend-ids": {{ 13788189, 27667188, 588943, 1574745, 5763893, 19661124, 45630528, 47078471, 42976078, 32943975 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2007-07-04"), "end-date": null } ] }
+{ "id": 9733942, "id-copy": 9733942, "alias": "Andra", "name": "AndraConrad", "user-since": datetime("2007-01-23T01:20:01.000Z"), "user-since-copy": datetime("2007-01-23T01:20:01.000Z"), "friend-ids": {{ 42791827, 36987912, 12650269, 5310067, 33419819, 36880069, 1146970, 20314, 10762565, 20657888, 31871678, 42279496, 9831201, 4223369, 46820320, 21703772, 1326858, 21739453, 20082273, 12950360 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2002-03-21"), "end-date": null } ] }
+{ "id": 9747652, "id-copy": 9747652, "alias": "Graham", "name": "GrahamGarratt", "user-since": datetime("2006-04-16T19:35:33.000Z"), "user-since-copy": datetime("2006-04-16T19:35:33.000Z"), "friend-ids": {{ 9995821, 7082678, 29813051, 33625501, 32785793, 23170533, 26581328, 35564866, 9147486, 17626916, 12721534, 22070579, 25749282, 27771492, 35217137, 6426437, 4217778, 6819045, 6410966, 43080321, 32112201, 20323505 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2005-09-26"), "end-date": null } ] }
+{ "id": 9783310, "id-copy": 9783310, "alias": "Basil", "name": "BasilLangston", "user-since": datetime("2005-06-10T11:35:51.000Z"), "user-since-copy": datetime("2005-06-10T11:35:51.000Z"), "friend-ids": {{ 21087606, 17287729, 8132136, 17055542, 5795845, 41180261, 10977404, 29700430, 47047119, 358942, 29290990, 19557422, 35447157, 33135473, 36720866, 39510564 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2000-05-11"), "end-date": date("2000-03-09") } ] }
+{ "id": 9819796, "id-copy": 9819796, "alias": "Emerson", "name": "EmersonWardle", "user-since": datetime("2006-08-20T20:22:11.000Z"), "user-since-copy": datetime("2006-08-20T20:22:11.000Z"), "friend-ids": {{ 5697147, 42936553, 12624322, 45309083, 10785774, 4176618 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2003-05-16"), "end-date": null } ] }
+{ "id": 9829834, "id-copy": 9829834, "alias": "Darryl", "name": "DarrylSullivan", "user-since": datetime("2011-07-24T00:12:33.000Z"), "user-since-copy": datetime("2011-07-24T00:12:33.000Z"), "friend-ids": {{ 8297654, 6071837, 27236382, 4657522, 9035310, 40427605, 2360931, 19796421, 7301200, 1264845, 12653555, 27518516 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2005-01-18"), "end-date": date("2010-05-20") } ] }
+{ "id": 9840013, "id-copy": 9840013, "alias": "Inger", "name": "IngerRuhl", "user-since": datetime("2009-05-27T20:14:42.000Z"), "user-since-copy": datetime("2009-05-27T20:14:42.000Z"), "friend-ids": {{ 36044692 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2010-09-15"), "end-date": null } ] }
+{ "id": 9879709, "id-copy": 9879709, "alias": "Winfred", "name": "WinfredCraig", "user-since": datetime("2005-08-03T19:34:00.000Z"), "user-since-copy": datetime("2005-08-03T19:34:00.000Z"), "friend-ids": {{ 22314477, 25116324, 22136373, 35942614, 21324680, 17967388, 29463891, 36125380, 20673052, 27353154, 25107580, 24689990, 17672337, 16922511, 26158336, 35966438, 26619840, 29808016, 12075922, 33292381, 17902188 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2010-02-04"), "end-date": null } ] }
+{ "id": 9880603, "id-copy": 9880603, "alias": "Davis", "name": "DavisRitter", "user-since": datetime("2009-12-18T18:55:46.000Z"), "user-since-copy": datetime("2009-12-18T18:55:46.000Z"), "friend-ids": {{ 10790833, 43529865, 23457220, 6745186, 22333440, 39380793, 2096806, 44121543, 29345888, 46499780, 31896682, 35084540, 6060378, 27402271, 18954641 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2002-01-11"), "end-date": null } ] }
+{ "id": 9919033, "id-copy": 9919033, "alias": "Bailey", "name": "BaileyHay", "user-since": datetime("2005-01-06T07:43:18.000Z"), "user-since-copy": datetime("2005-01-06T07:43:18.000Z"), "friend-ids": {{ 28198532 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2009-02-08"), "end-date": date("2010-06-08") } ] }
+{ "id": 9931588, "id-copy": 9931588, "alias": "Sheri", "name": "SheriHindman", "user-since": datetime("2011-02-19T03:55:37.000Z"), "user-since-copy": datetime("2011-02-19T03:55:37.000Z"), "friend-ids": {{ 10993709, 28005344, 31884585, 1581885, 46332238, 47401902, 38814902, 39736365, 24318394, 15329318, 35794552, 14913021, 8723328, 28102869, 27218765, 21310255 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2011-08-17"), "end-date": date("2011-12-15") } ] }
+{ "id": 9952342, "id-copy": 9952342, "alias": "Christal", "name": "ChristalMcmichaels", "user-since": datetime("2008-02-13T13:25:45.000Z"), "user-since-copy": datetime("2008-02-13T13:25:45.000Z"), "friend-ids": {{ 12290348, 1563117, 10883525, 17285406, 3798829, 3734533, 13084348, 31001579, 23655942, 44480002, 11803789, 8240833, 42718608, 41919526, 37582304, 10494964, 10815416, 10676699, 9376307 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2011-05-16"), "end-date": null } ] }
+{ "id": 9962236, "id-copy": 9962236, "alias": "Craig", "name": "CraigKight", "user-since": datetime("2010-02-15T15:58:03.000Z"), "user-since-copy": datetime("2010-02-15T15:58:03.000Z"), "friend-ids": {{ 45604304, 40911167, 39517053, 6912584, 898627, 8412812, 33530827, 30135549, 14762146, 46313211, 21143796, 39820220, 11462372, 23575315 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2001-02-05"), "end-date": date("2008-01-04") } ] }
+{ "id": 9979750, "id-copy": 9979750, "alias": "Reginald", "name": "ReginaldAltman", "user-since": datetime("2007-04-04T08:51:58.000Z"), "user-since-copy": datetime("2007-04-04T08:51:58.000Z"), "friend-ids": {{ 2988287 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2002-01-28"), "end-date": null } ] }
+{ "id": 9986206, "id-copy": 9986206, "alias": "Tatiana", "name": "TatianaAlbright", "user-since": datetime("2006-03-21T10:00:55.000Z"), "user-since-copy": datetime("2006-03-21T10:00:55.000Z"), "friend-ids": {{ 42869099, 40178170, 13922993, 28844962, 26206785, 41293581, 17131809, 1583964, 47236558, 2656158, 11008100, 3994698, 23764118, 14275676, 4922979, 28466879, 16454954, 3620561, 42044685, 12665882, 18354684 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2012-04-24"), "end-date": null } ] }
+{ "id": 9988417, "id-copy": 9988417, "alias": "Coline", "name": "ColineLane", "user-since": datetime("2010-01-01T00:12:39.000Z"), "user-since-copy": datetime("2010-01-01T00:12:39.000Z"), "friend-ids": {{ 17656229, 42804152 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2012-05-01"), "end-date": null } ] }
+{ "id": 9993001, "id-copy": 9993001, "alias": "Herbie", "name": "HerbieStall", "user-since": datetime("2010-06-14T03:01:11.000Z"), "user-since-copy": datetime("2010-06-14T03:01:11.000Z"), "friend-ids": {{ 12003033, 40923715, 34166285, 47927261, 638933, 17338590 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2009-07-12"), "end-date": null } ] }
+{ "id": 9996817, "id-copy": 9996817, "alias": "Vere", "name": "VereWilkerson", "user-since": datetime("2012-02-05T22:05:44.000Z"), "user-since-copy": datetime("2012-02-05T22:05:44.000Z"), "friend-ids": {{ 30010110, 31604568, 5741065, 29161468, 22429704, 16954129, 26525860, 1490181, 11444321, 24455724, 10411850, 39851031, 16059860, 32050795, 13116007, 12071588 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2004-11-04"), "end-date": null } ] }
+{ "id": 10001410, "id-copy": 10001410, "alias": "Denzil", "name": "DenzilLedgerwood", "user-since": datetime("2006-12-24T10:56:58.000Z"), "user-since-copy": datetime("2006-12-24T10:56:58.000Z"), "friend-ids": {{ 25633920, 39748697, 3557647, 44396047, 25225495, 38723684, 5854330 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2000-08-14"), "end-date": date("2011-07-20") } ] }
+{ "id": 10002907, "id-copy": 10002907, "alias": "Maegan", "name": "MaeganErschoff", "user-since": datetime("2011-10-15T18:08:56.000Z"), "user-since-copy": datetime("2011-10-15T18:08:56.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2009-02-06"), "end-date": date("2011-05-20") } ] }
+{ "id": 10054327, "id-copy": 10054327, "alias": "Poppy", "name": "PoppyKellogg", "user-since": datetime("2010-03-28T09:43:49.000Z"), "user-since-copy": datetime("2010-03-28T09:43:49.000Z"), "friend-ids": {{ 10785684, 26545687, 942400, 18147517, 12133643, 17848751, 40864121, 18975370, 26159158, 42348235, 21795276, 40155922, 35240759 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2012-03-24"), "end-date": null } ] }
+{ "id": 10065250, "id-copy": 10065250, "alias": "Debbie", "name": "DebbieBrinigh", "user-since": datetime("2012-01-05T15:05:48.000Z"), "user-since-copy": datetime("2012-01-05T15:05:48.000Z"), "friend-ids": {{ 23794420, 31166549, 3372724, 35955226, 45241312, 33488036, 17353508, 10094234, 12751868 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2000-06-28"), "end-date": date("2005-06-03") } ] }
+{ "id": 10073002, "id-copy": 10073002, "alias": "Josefa", "name": "JosefaNewman", "user-since": datetime("2010-10-06T09:28:29.000Z"), "user-since-copy": datetime("2010-10-06T09:28:29.000Z"), "friend-ids": {{ 7549910, 7287709, 24063891, 41208589, 22325854, 16465930, 45067165, 42784968, 26414870, 16479308, 22681119, 40811475, 9603161, 23525416, 15131604, 4782290, 36997646, 35862360, 42008502, 438438, 25913601, 39300786, 15041382, 37410001 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2011-02-05"), "end-date": date("2011-10-24") } ] }
+{ "id": 10079965, "id-copy": 10079965, "alias": "Mason", "name": "MasonReamer", "user-since": datetime("2008-08-10T02:16:36.000Z"), "user-since-copy": datetime("2008-08-10T02:16:36.000Z"), "friend-ids": {{ 37149190, 37736572, 35955709, 28586597, 45460389 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2002-01-18"), "end-date": date("2010-12-09") } ] }
+{ "id": 10085446, "id-copy": 10085446, "alias": "Merla", "name": "MerlaWhitehead", "user-since": datetime("2006-12-08T11:13:30.000Z"), "user-since-copy": datetime("2006-12-08T11:13:30.000Z"), "friend-ids": {{ 44039547 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-03-16"), "end-date": date("2009-04-16") } ] }
+{ "id": 10089976, "id-copy": 10089976, "alias": "Marion", "name": "MarionThomlinson", "user-since": datetime("2006-06-27T14:11:49.000Z"), "user-since-copy": datetime("2006-06-27T14:11:49.000Z"), "friend-ids": {{ 39404598, 46190974, 43413339, 41250692, 4194349, 5150083, 35574492, 30896673, 15969653, 41889132, 38801872, 17834003, 42587459, 42269051, 20206793, 46257713, 2735409, 28567746, 6641216, 3627253, 15945805, 33861471, 9997931, 38242090 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2011-11-22"), "end-date": date("2011-06-01") } ] }
+{ "id": 10090042, "id-copy": 10090042, "alias": "Gaye", "name": "GayeHayhurst", "user-since": datetime("2006-09-23T14:26:31.000Z"), "user-since-copy": datetime("2006-09-23T14:26:31.000Z"), "friend-ids": {{ 41099035, 16443590, 9899624, 2459064, 25428448, 1420220, 1487058, 13700561, 11008052, 36459693, 45632468, 30351729, 33053870, 26372759, 10801940, 37166367 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2005-07-15"), "end-date": date("2010-05-04") } ] }
+{ "id": 10100707, "id-copy": 10100707, "alias": "Brittni", "name": "BrittniEaster", "user-since": datetime("2008-10-03T02:27:48.000Z"), "user-since-copy": datetime("2008-10-03T02:27:48.000Z"), "friend-ids": {{ 28725707, 8497950, 18892135, 1016149, 32023719, 34079976, 39582966, 15469248, 14059091, 6681733, 18398487, 41385960 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2006-04-21"), "end-date": null } ] }
+{ "id": 10122346, "id-copy": 10122346, "alias": "Salal", "name": "SalalPearson", "user-since": datetime("2011-11-14T10:42:11.000Z"), "user-since-copy": datetime("2011-11-14T10:42:11.000Z"), "friend-ids": {{ 44003884, 37124809, 7600567, 5158911, 31009406, 10708460 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2001-02-17"), "end-date": date("2010-06-23") } ] }
+{ "id": 10123051, "id-copy": 10123051, "alias": "Rowland", "name": "RowlandWaldron", "user-since": datetime("2011-08-01T17:20:14.000Z"), "user-since-copy": datetime("2011-08-01T17:20:14.000Z"), "friend-ids": {{ 7693849, 5416143, 10885197, 39771258, 41278769, 16236783, 18739058, 2293485, 32013369, 34882536, 14339467, 3680575, 4461977, 33715303, 26345760, 45729149, 17585375, 39496021 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2001-12-10"), "end-date": date("2006-04-07") } ] }
+{ "id": 10133458, "id-copy": 10133458, "alias": "Kati", "name": "KatiPennington", "user-since": datetime("2011-01-28T10:51:37.000Z"), "user-since-copy": datetime("2011-01-28T10:51:37.000Z"), "friend-ids": {{ 41299906, 11523198, 8344474, 36086944, 34330342, 43585884, 6751565, 23415221, 32275829, 43645200 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2005-11-11"), "end-date": null } ] }
+{ "id": 10138039, "id-copy": 10138039, "alias": "Farah", "name": "FarahAnn", "user-since": datetime("2008-05-10T19:04:28.000Z"), "user-since-copy": datetime("2008-05-10T19:04:28.000Z"), "friend-ids": {{ 32501277, 13715476, 10452566, 2652600, 16449577, 12508457, 30925424, 21595197, 26030962, 31683678 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2003-10-02"), "end-date": null } ] }
+{ "id": 10148251, "id-copy": 10148251, "alias": "Ghislaine", "name": "GhislaineFowler", "user-since": datetime("2005-12-08T05:25:56.000Z"), "user-since-copy": datetime("2005-12-08T05:25:56.000Z"), "friend-ids": {{ 14692731, 29964772 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2008-12-27"), "end-date": date("2008-04-02") } ] }
+{ "id": 10185346, "id-copy": 10185346, "alias": "Noah", "name": "NoahAshmore", "user-since": datetime("2006-04-04T14:33:43.000Z"), "user-since-copy": datetime("2006-04-04T14:33:43.000Z"), "friend-ids": {{ 15819384, 46052301, 7102428, 7977240, 30337629, 31480307, 30013142, 4192580, 34814572, 6841517, 2253788, 31150059, 505825, 27897490, 11402219 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2012-06-06"), "end-date": null } ] }
+{ "id": 10205539, "id-copy": 10205539, "alias": "Raeburn", "name": "RaeburnWire", "user-since": datetime("2007-04-28T23:05:24.000Z"), "user-since-copy": datetime("2007-04-28T23:05:24.000Z"), "friend-ids": {{ 13609724, 40251506 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2000-08-09"), "end-date": null } ] }
+{ "id": 10211827, "id-copy": 10211827, "alias": "Fanny", "name": "FannyHarrold", "user-since": datetime("2010-08-28T09:57:52.000Z"), "user-since-copy": datetime("2010-08-28T09:57:52.000Z"), "friend-ids": {{ 4061493, 30492642, 8550070, 34805906, 5798646, 39169853, 45190690, 34218456, 3758565, 18038216 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2008-10-14"), "end-date": date("2008-05-18") } ] }
+{ "id": 10212385, "id-copy": 10212385, "alias": "Alice", "name": "AliceJones", "user-since": datetime("2009-05-16T16:08:03.000Z"), "user-since-copy": datetime("2009-05-16T16:08:03.000Z"), "friend-ids": {{ 4158604, 3204211, 21491737, 39619715, 9750334 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2012-04-19"), "end-date": null } ] }
+{ "id": 10222144, "id-copy": 10222144, "alias": "Alvina", "name": "AlvinaTanner", "user-since": datetime("2007-10-15T04:24:14.000Z"), "user-since-copy": datetime("2007-10-15T04:24:14.000Z"), "friend-ids": {{ 44207447, 29837430, 407059, 4562324, 970458, 31348025, 16439061, 13011150, 23510630, 21529259, 8279487, 28052530, 36551405, 17492050, 17983056, 11834104, 242520, 9279232, 4179609, 28407763, 23038009, 36977762, 8779957, 15040402 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2006-05-27"), "end-date": null } ] }
+{ "id": 10224400, "id-copy": 10224400, "alias": "Malvina", "name": "MalvinaPery", "user-since": datetime("2009-01-25T03:41:22.000Z"), "user-since-copy": datetime("2009-01-25T03:41:22.000Z"), "friend-ids": {{ 17095877, 17062955, 13129292, 31635980, 32747924, 902714, 32032985, 44944935, 30544897, 44429244 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2001-01-11"), "end-date": date("2011-04-10") } ] }
+{ "id": 10269739, "id-copy": 10269739, "alias": "Shantel", "name": "ShantelEve", "user-since": datetime("2012-06-06T00:37:05.000Z"), "user-since-copy": datetime("2012-06-06T00:37:05.000Z"), "friend-ids": {{ 39436396, 20382971, 47821933, 28867521, 23217564, 40672635, 34693766, 4383592, 42534606, 23535312, 9112260, 4828073, 37429286, 27965200, 30257544, 47609429, 18527025, 33339218, 898986, 2817270, 6040099, 47802547 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2000-05-24"), "end-date": null } ] }
+{ "id": 10270597, "id-copy": 10270597, "alias": "Ava", "name": "AvaTanner", "user-since": datetime("2010-04-23T11:49:39.000Z"), "user-since-copy": datetime("2010-04-23T11:49:39.000Z"), "friend-ids": {{ 38894360, 9403074, 25855965, 36511208, 4947767, 10318201, 3532083, 28684767, 22730535, 17994309, 21209113, 14980333, 5611975, 31951870, 16697364, 5033131, 13637894, 18107216, 9769275, 25479923, 15320268, 28897820, 22865104 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2011-04-12"), "end-date": date("2011-09-07") } ] }
+{ "id": 10278550, "id-copy": 10278550, "alias": "Parker", "name": "ParkerWinton", "user-since": datetime("2008-03-02T18:54:35.000Z"), "user-since-copy": datetime("2008-03-02T18:54:35.000Z"), "friend-ids": {{ 281420, 13481584, 25554653, 2922131, 15313837, 33567564, 20182917, 20143660, 35884326, 22038516, 183180 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2002-12-16"), "end-date": date("2010-08-04") } ] }
+{ "id": 10280533, "id-copy": 10280533, "alias": "Normand", "name": "NormandAckerley", "user-since": datetime("2008-05-18T00:44:35.000Z"), "user-since-copy": datetime("2008-05-18T00:44:35.000Z"), "friend-ids": {{ 46908522, 2002203, 15632192, 3790633, 21300428, 15452344, 34478785, 18864214, 32842683, 10486268, 2496859 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2010-12-07"), "end-date": null } ] }
+{ "id": 10283503, "id-copy": 10283503, "alias": "Terrilyn", "name": "TerrilynZadovsky", "user-since": datetime("2007-06-17T05:40:01.000Z"), "user-since-copy": datetime("2007-06-17T05:40:01.000Z"), "friend-ids": {{ 30185148, 22395650, 3212998, 41571861, 21336440, 41050091 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2009-12-14"), "end-date": null } ] }
+{ "id": 10287028, "id-copy": 10287028, "alias": "Wilfred", "name": "WilfredChurchill", "user-since": datetime("2007-08-01T14:14:25.000Z"), "user-since-copy": datetime("2007-08-01T14:14:25.000Z"), "friend-ids": {{ 38355737, 39891840, 41036196, 39165706, 1155288, 15280633, 9744287, 11567914, 11225763, 2297894, 14386027, 67174, 28097703, 28721858, 6504409, 6743503, 22860419, 17773814, 34697084, 5419586, 45771084 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2002-08-08"), "end-date": null } ] }
+{ "id": 10317160, "id-copy": 10317160, "alias": "Maria", "name": "MariaHair", "user-since": datetime("2006-05-21T16:06:00.000Z"), "user-since-copy": datetime("2006-05-21T16:06:00.000Z"), "friend-ids": {{ 7063473, 43027344, 2119671, 39231388, 34041933, 5141408, 20278936 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2005-10-20"), "end-date": null } ] }
+{ "id": 10322023, "id-copy": 10322023, "alias": "Shanita", "name": "ShanitaBeedell", "user-since": datetime("2011-06-09T23:50:09.000Z"), "user-since-copy": datetime("2011-06-09T23:50:09.000Z"), "friend-ids": {{ 22628842, 2169935, 20656034, 9086684, 17234788, 11936164, 12465122, 2543006, 40067557, 36767662, 633930, 41805132, 13246529, 43801547, 44953975, 36902947, 34935791, 22923033, 28190533, 18230134, 9484458, 21184932 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2011-10-10"), "end-date": null } ] }
+{ "id": 10346116, "id-copy": 10346116, "alias": "Breana", "name": "BreanaPainter", "user-since": datetime("2012-04-05T12:15:17.000Z"), "user-since-copy": datetime("2012-04-05T12:15:17.000Z"), "friend-ids": {{ 39999376, 5382299, 36254541, 16829210, 7084172, 13545656, 24681698, 34171417, 28514693, 8090159, 35046661, 44544921, 47754565, 28732689, 19680056, 21398367, 39260450 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2012-08-24"), "end-date": date("2012-08-24") } ] }
+{ "id": 10361965, "id-copy": 10361965, "alias": "Arlen", "name": "ArlenFlick", "user-since": datetime("2011-07-14T18:38:37.000Z"), "user-since-copy": datetime("2011-07-14T18:38:37.000Z"), "friend-ids": {{ 34249140, 2887282, 47622716, 3897801, 33692288, 14374380, 14183995, 41311739, 6378075, 17721901, 20807501, 8908974, 41080464, 26497672 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2008-05-18"), "end-date": date("2011-09-18") } ] }
+{ "id": 10380031, "id-copy": 10380031, "alias": "Otha", "name": "OthaHaines", "user-since": datetime("2005-08-08T04:10:50.000Z"), "user-since-copy": datetime("2005-08-08T04:10:50.000Z"), "friend-ids": {{ 2710866, 28894512, 36379679, 32545673, 38671874, 16746916, 39103475, 19783615, 17514492, 42617267, 7461114, 17712393, 43474200, 3806350, 5065542, 35722940 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2011-05-07"), "end-date": null } ] }
+{ "id": 10391179, "id-copy": 10391179, "alias": "Raymond", "name": "RaymondHoopengarner", "user-since": datetime("2006-04-06T18:32:20.000Z"), "user-since-copy": datetime("2006-04-06T18:32:20.000Z"), "friend-ids": {{ 35664656, 36940003, 35836359, 25322876, 45895708, 14553421 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2005-08-05"), "end-date": date("2007-01-09") } ] }
+{ "id": 10398562, "id-copy": 10398562, "alias": "Brendon", "name": "BrendonMaclagan", "user-since": datetime("2012-02-23T06:18:49.000Z"), "user-since-copy": datetime("2012-02-23T06:18:49.000Z"), "friend-ids": {{ 39206829, 37980663, 36889290, 9114653, 26448451, 15142055, 23349234, 11668644, 22072984, 2091972, 957976, 26110137, 20947598, 32127830, 35850034, 39029675, 21265582, 26725192, 13963111, 4392994, 37042547 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2001-09-21"), "end-date": null } ] }
+{ "id": 10405423, "id-copy": 10405423, "alias": "Pauletta", "name": "PaulettaGuess", "user-since": datetime("2007-06-11T02:54:36.000Z"), "user-since-copy": datetime("2007-06-11T02:54:36.000Z"), "friend-ids": {{ 14845791, 24263161, 2648994, 30766767, 10127359, 20706390 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2002-10-27"), "end-date": null } ] }
+{ "id": 10479073, "id-copy": 10479073, "alias": "Rhianna", "name": "RhiannaWerry", "user-since": datetime("2009-09-17T19:42:47.000Z"), "user-since-copy": datetime("2009-09-17T19:42:47.000Z"), "friend-ids": {{ 30293616, 42971604, 8411318, 37648744, 27412687, 17821200, 45008072 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2004-11-02"), "end-date": date("2011-06-24") } ] }
+{ "id": 10479190, "id-copy": 10479190, "alias": "Carmine", "name": "CarmineMortland", "user-since": datetime("2011-06-18T02:57:13.000Z"), "user-since-copy": datetime("2011-06-18T02:57:13.000Z"), "friend-ids": {{ 36090597, 35550849, 19614765, 34665409, 7740163, 12824683, 12997403, 32586142, 10137983, 44900811, 30392212, 43177710, 47792212 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2002-02-23"), "end-date": null } ] }
+{ "id": 10601758, "id-copy": 10601758, "alias": "Blossom", "name": "BlossomClark", "user-since": datetime("2011-08-16T23:44:16.000Z"), "user-since-copy": datetime("2011-08-16T23:44:16.000Z"), "friend-ids": {{ 22624576, 6945784, 47816004, 8072206, 23953052, 22668193, 8668574, 2269602, 39137309, 38996903, 23516086, 31166264, 28322741, 46296094, 36547681, 7287738, 15727604, 13556387, 2624138 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2004-10-15"), "end-date": date("2008-07-17") } ] }
+{ "id": 10602166, "id-copy": 10602166, "alias": "Karine", "name": "KarineAdams", "user-since": datetime("2006-03-03T20:36:12.000Z"), "user-since-copy": datetime("2006-03-03T20:36:12.000Z"), "friend-ids": {{ 4463206, 23962283, 34321170, 10546383, 39886106, 37478996 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2000-10-28"), "end-date": date("2010-04-26") } ] }
+{ "id": 10613617, "id-copy": 10613617, "alias": "Jeanie", "name": "JeanieEiford", "user-since": datetime("2007-02-09T12:16:09.000Z"), "user-since-copy": datetime("2007-02-09T12:16:09.000Z"), "friend-ids": {{ 24843944, 3651507, 25077638, 18662161, 46723847, 31558857, 11235682, 15640606, 31889112, 45342233, 25865191, 1530020, 39187188, 4939030, 19220487, 19619126, 25284665, 1206869, 40740763 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2001-07-22"), "end-date": null } ] }
+{ "id": 10640851, "id-copy": 10640851, "alias": "Tabitha", "name": "TabithaWhitten", "user-since": datetime("2010-01-28T14:25:58.000Z"), "user-since-copy": datetime("2010-01-28T14:25:58.000Z"), "friend-ids": {{ 42792549, 5330514, 24582133, 43384590, 38083439, 31221232, 18064537, 21736064, 7919520, 18998284, 20165148, 28492287, 21987533, 23638155 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2006-06-18"), "end-date": date("2007-07-20") } ] }
+{ "id": 10642153, "id-copy": 10642153, "alias": "Wally", "name": "WallyRiggle", "user-since": datetime("2011-10-10T21:43:33.000Z"), "user-since-copy": datetime("2011-10-10T21:43:33.000Z"), "friend-ids": {{ 32910135, 45556839, 6526394, 13177451, 10588491, 40270322, 17438379, 21204776, 46036116, 44249789, 7375979, 43487252, 24858016, 3947997 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2001-10-10"), "end-date": null } ] }
+{ "id": 10663741, "id-copy": 10663741, "alias": "Gaylord", "name": "GaylordWynne", "user-since": datetime("2007-09-07T09:15:35.000Z"), "user-since-copy": datetime("2007-09-07T09:15:35.000Z"), "friend-ids": {{ 34508923, 28228552, 7714885, 16525247, 30914675, 8152699, 26553788, 8070452, 45739728 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2006-05-18"), "end-date": date("2008-04-07") } ] }
+{ "id": 10671115, "id-copy": 10671115, "alias": "Montague", "name": "MontagueLangston", "user-since": datetime("2007-09-20T00:32:15.000Z"), "user-since-copy": datetime("2007-09-20T00:32:15.000Z"), "friend-ids": {{ 18236000, 47490167, 40246549, 25232933, 22604487, 36974958, 44747862, 2137180, 39244601, 39608406, 23319330, 21166788, 21726220, 12703943, 36564459, 8379538, 43010567, 24538004, 173522, 6132291, 21199763, 26285128, 2350066 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2001-01-07"), "end-date": null } ] }
+{ "id": 10678567, "id-copy": 10678567, "alias": "Detta", "name": "DettaIronmonger", "user-since": datetime("2006-05-01T08:52:26.000Z"), "user-since-copy": datetime("2006-05-01T08:52:26.000Z"), "friend-ids": {{ 11098679, 15763619, 12715761, 10175990, 43581466, 4595173, 17163835, 44918467, 38256765, 13239047, 25476309, 9075112, 19581524, 46478013, 24168854, 34121818, 25604978, 21114089 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2007-06-08"), "end-date": null } ] }
+{ "id": 10708477, "id-copy": 10708477, "alias": "Zacharias", "name": "ZachariasRandolph", "user-since": datetime("2008-07-13T16:12:33.000Z"), "user-since-copy": datetime("2008-07-13T16:12:33.000Z"), "friend-ids": {{ 18251027, 47694844, 25569678, 33130234, 7351010, 32617025, 40619749, 28576965, 34970660, 34320919, 17056847, 46007935, 244756, 3130710, 5218614, 6968874, 19440356, 448790, 3336700, 44725864, 24738046, 6159443, 14380294, 20289778 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2007-09-09"), "end-date": null } ] }
+{ "id": 10721059, "id-copy": 10721059, "alias": "Amandine", "name": "AmandineRockwell", "user-since": datetime("2008-09-24T21:50:39.000Z"), "user-since-copy": datetime("2008-09-24T21:50:39.000Z"), "friend-ids": {{ 10360854, 15197739, 28812340, 12172446, 9354363, 23580760, 6364957, 20048548 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2003-12-06"), "end-date": null } ] }
+{ "id": 10733617, "id-copy": 10733617, "alias": "Leonardo", "name": "LeonardoKight", "user-since": datetime("2008-10-20T17:30:29.000Z"), "user-since-copy": datetime("2008-10-20T17:30:29.000Z"), "friend-ids": {{ 39687903, 7235506, 34696496, 25995345, 18435380, 47473591, 15710408, 44232442, 39520147, 36384026, 25160887, 245860, 1195579, 4587411, 536916, 47052672, 33953823, 13203710 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2007-07-12"), "end-date": date("2010-03-16") } ] }
+{ "id": 10738477, "id-copy": 10738477, "alias": "Kenith", "name": "KenithLeichter", "user-since": datetime("2012-07-10T15:21:51.000Z"), "user-since-copy": datetime("2012-07-10T15:21:51.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2006-07-28"), "end-date": date("2009-06-03") } ] }
+{ "id": 10745200, "id-copy": 10745200, "alias": "Kaety", "name": "KaetyOppenheimer", "user-since": datetime("2008-11-21T08:11:11.000Z"), "user-since-copy": datetime("2008-11-21T08:11:11.000Z"), "friend-ids": {{ 32006369, 4542624, 28242708, 20936957, 11063561, 31392192, 34444041, 754368, 37317926 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2010-06-07"), "end-date": null } ] }
+{ "id": 10749553, "id-copy": 10749553, "alias": "Rolland", "name": "RollandMunshower", "user-since": datetime("2005-12-26T19:26:32.000Z"), "user-since-copy": datetime("2005-12-26T19:26:32.000Z"), "friend-ids": {{ 27080985, 4355429, 17027260, 30203290, 37292858, 1935550, 467329, 24265915, 4926329, 28586308, 27299677, 25356918, 14171255, 319307, 15014794 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2011-04-21"), "end-date": null } ] }
+{ "id": 10751260, "id-copy": 10751260, "alias": "Chrysanta", "name": "ChrysantaSanforth", "user-since": datetime("2009-06-02T12:54:32.000Z"), "user-since-copy": datetime("2009-06-02T12:54:32.000Z"), "friend-ids": {{ 6064707, 44017707, 22957433, 38426343, 24694205, 1061085, 24827089, 12192854, 40718843 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2011-01-19"), "end-date": date("2011-10-02") } ] }
+{ "id": 10765090, "id-copy": 10765090, "alias": "Louiza", "name": "LouizaMcelroy", "user-since": datetime("2012-08-14T02:46:00.000Z"), "user-since-copy": datetime("2012-08-14T02:46:00.000Z"), "friend-ids": {{ 14365973, 9091111, 44279279, 45125689, 29955385, 23874606, 18142514, 24878700, 13928633, 47391704, 29729670, 35422059, 987030, 3200788, 7640346, 32947024, 32550247, 25746061, 34112521, 41193622, 2620213, 30090329, 5531715 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2002-06-16"), "end-date": date("2003-05-13") } ] }
+{ "id": 10771030, "id-copy": 10771030, "alias": "Jen", "name": "JenZaun", "user-since": datetime("2006-12-02T14:42:43.000Z"), "user-since-copy": datetime("2006-12-02T14:42:43.000Z"), "friend-ids": {{ 38166077 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2003-12-01"), "end-date": date("2010-04-12") } ] }
+{ "id": 10779373, "id-copy": 10779373, "alias": "Donya", "name": "DonyaWegley", "user-since": datetime("2012-03-28T01:26:06.000Z"), "user-since-copy": datetime("2012-03-28T01:26:06.000Z"), "friend-ids": {{ 24977052, 19856115, 36795249, 7875698, 23317261, 5916235, 17789989, 41932923 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2008-10-18"), "end-date": null } ] }
+{ "id": 10783822, "id-copy": 10783822, "alias": "Emerald", "name": "EmeraldMillard", "user-since": datetime("2008-08-07T16:33:44.000Z"), "user-since-copy": datetime("2008-08-07T16:33:44.000Z"), "friend-ids": {{ 22464360, 7890894, 18256597, 33659179, 24554534, 30962087, 29716339, 23689397, 45113518, 19997635 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2001-06-10"), "end-date": date("2006-12-02") } ] }
+{ "id": 10797166, "id-copy": 10797166, "alias": "Alethea", "name": "AletheaMills", "user-since": datetime("2011-01-10T03:06:16.000Z"), "user-since-copy": datetime("2011-01-10T03:06:16.000Z"), "friend-ids": {{ 25077851, 2396037, 25762626, 31358162, 41492027, 31211140, 38478662, 9688210, 16865534, 4209161, 19863828, 23760993, 36041139, 46184667 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2004-05-04"), "end-date": null } ] }
+{ "id": 10799674, "id-copy": 10799674, "alias": "Dolores", "name": "DoloresPolson", "user-since": datetime("2006-03-24T00:54:47.000Z"), "user-since-copy": datetime("2006-03-24T00:54:47.000Z"), "friend-ids": {{ 40482317, 21393644, 151122, 13958566, 6524741, 1269094, 34703787, 38215473, 20258639, 144407, 23903205, 46922014, 26741209, 34932062, 1043581, 14090176, 45243069, 19226320, 33271281, 20215000, 46383495, 42405679, 42360649 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2009-07-18"), "end-date": null } ] }
+{ "id": 10809322, "id-copy": 10809322, "alias": "Alden", "name": "AldenHiggens", "user-since": datetime("2011-02-06T01:31:58.000Z"), "user-since-copy": datetime("2011-02-06T01:31:58.000Z"), "friend-ids": {{ 44750450, 24564153, 42513064, 33316253, 21036452, 27132567, 29231674, 18040424, 36564417, 17474605, 14126628, 18988855, 35594147, 35685289, 40967850 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2002-09-26"), "end-date": null } ] }
+{ "id": 10824484, "id-copy": 10824484, "alias": "Linda", "name": "LindaStanfield", "user-since": datetime("2009-03-03T12:54:55.000Z"), "user-since-copy": datetime("2009-03-03T12:54:55.000Z"), "friend-ids": {{ 39164563, 20321780, 19901289, 37969494, 15051354, 42576590, 14550253, 33649901, 6008727, 17749643, 7792769, 18652053, 8565400, 43899372, 7433016, 42506713 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2012-03-21"), "end-date": null } ] }
+{ "id": 10832305, "id-copy": 10832305, "alias": "Briony", "name": "BrionyBaldwin", "user-since": datetime("2011-03-03T22:00:38.000Z"), "user-since-copy": datetime("2011-03-03T22:00:38.000Z"), "friend-ids": {{ 20436897, 36519715, 35325917, 31686319, 2644929, 3401668, 39344422, 18601722, 40274111, 30032679, 9312830, 5581755, 41164101, 35883066, 8274432, 4315219, 26200418, 43810182, 44718149, 6387153, 43086214, 39558538, 36036905, 25715671 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2007-01-21"), "end-date": date("2008-02-25") } ] }
+{ "id": 10851595, "id-copy": 10851595, "alias": "Juan", "name": "JuanSoames", "user-since": datetime("2006-02-16T05:34:28.000Z"), "user-since-copy": datetime("2006-02-16T05:34:28.000Z"), "friend-ids": {{ 34589906, 8801547, 38357163, 39649840, 18254469, 38911658, 17825991, 26015024, 29742264, 13155934, 28459597, 34931012, 20376527 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2008-11-17"), "end-date": date("2009-01-13") } ] }
+{ "id": 10858909, "id-copy": 10858909, "alias": "Kiley", "name": "KileyCoates", "user-since": datetime("2011-02-03T03:12:41.000Z"), "user-since-copy": datetime("2011-02-03T03:12:41.000Z"), "friend-ids": {{ 47990206, 29775839, 33872749, 38952297, 38802567, 38822660, 12420330, 18852873, 30468156, 29085185, 2660660, 28283210, 6711584, 35851765, 31124383, 39930865, 18329720 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2006-09-22"), "end-date": null } ] }
+{ "id": 10861183, "id-copy": 10861183, "alias": "Zilla", "name": "ZillaOneal", "user-since": datetime("2008-03-12T23:37:18.000Z"), "user-since-copy": datetime("2008-03-12T23:37:18.000Z"), "friend-ids": {{ 26262188, 17172669, 43068853, 47767064, 34552281, 33602720, 35448839, 6347557, 11913432, 45186875, 10451537, 46881437, 27965706 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2008-09-03"), "end-date": date("2009-07-22") } ] }
+{ "id": 10865788, "id-copy": 10865788, "alias": "Ebba", "name": "EbbaSwartzbaugh", "user-since": datetime("2007-08-18T11:38:20.000Z"), "user-since-copy": datetime("2007-08-18T11:38:20.000Z"), "friend-ids": {{ 12850265, 19824056, 2754383, 43333892, 9287993, 14972999, 3729396, 20735424 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2001-10-07"), "end-date": date("2004-07-17") } ] }
+{ "id": 10889389, "id-copy": 10889389, "alias": "Roselyn", "name": "RoselynLlora", "user-since": datetime("2012-03-25T15:21:06.000Z"), "user-since-copy": datetime("2012-03-25T15:21:06.000Z"), "friend-ids": {{ 38921827, 1378686, 22284385, 17464785, 16302500, 47598267, 25016712, 11151378, 16381115, 16371401 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2005-12-02"), "end-date": null } ] }
+{ "id": 10894411, "id-copy": 10894411, "alias": "Lacy", "name": "LacyShaw", "user-since": datetime("2006-04-06T00:11:24.000Z"), "user-since-copy": datetime("2006-04-06T00:11:24.000Z"), "friend-ids": {{ 4203591, 28370134, 5239468, 12951448, 39355113, 9126812, 5662652, 4633221, 11954172, 33269236, 11545355, 14018236, 21980886, 34750979, 22877356 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2009-04-12"), "end-date": null } ] }
+{ "id": 10896556, "id-copy": 10896556, "alias": "Kimberleigh", "name": "KimberleighWoolery", "user-since": datetime("2005-05-12T17:22:37.000Z"), "user-since-copy": datetime("2005-05-12T17:22:37.000Z"), "friend-ids": {{ 6300953, 46149018, 25478406, 577782, 38073266, 11461118, 10240145, 686269, 37990652, 26865957 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2007-05-03"), "end-date": null } ] }
+{ "id": 10901047, "id-copy": 10901047, "alias": "Salvador", "name": "SalvadorBynum", "user-since": datetime("2012-01-13T02:30:17.000Z"), "user-since-copy": datetime("2012-01-13T02:30:17.000Z"), "friend-ids": {{ 29122263, 27975257, 7988516, 9270552, 17837898, 42339445, 46097101, 32303800, 17233223, 10656090, 36709955, 17535336, 27157992, 30360627, 15304415, 28922979, 27243261, 9307382, 43171015, 31593421, 21246902, 40452339, 25735551, 23716187 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2010-11-27"), "end-date": null } ] }
+{ "id": 10905802, "id-copy": 10905802, "alias": "Jamika", "name": "JamikaJowers", "user-since": datetime("2007-05-24T01:31:04.000Z"), "user-since-copy": datetime("2007-05-24T01:31:04.000Z"), "friend-ids": {{ 16476991, 9041491, 10867973, 18057276, 13716912, 184635, 47717267, 37995364 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2008-08-20"), "end-date": null } ] }
+{ "id": 10925071, "id-copy": 10925071, "alias": "Gil", "name": "GilFocell", "user-since": datetime("2005-11-08T20:28:01.000Z"), "user-since-copy": datetime("2005-11-08T20:28:01.000Z"), "friend-ids": {{ 9416716, 42743353, 43396785, 44271346, 32924780, 44752785, 19741326, 39315503, 25154503, 29170056, 15457515, 14764269, 47861907, 15230067, 15326613, 6336542, 44127013, 1048087, 34624221, 19951452, 12778135 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2009-01-07"), "end-date": null } ] }
+{ "id": 10931563, "id-copy": 10931563, "alias": "Laraine", "name": "LaraineCountryman", "user-since": datetime("2012-03-17T17:06:59.000Z"), "user-since-copy": datetime("2012-03-17T17:06:59.000Z"), "friend-ids": {{ 17266368, 75990, 37678426, 43207424, 37434492, 26338447, 33450799, 5401110, 44962643, 5514847 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2008-09-08"), "end-date": null } ] }
+{ "id": 10951918, "id-copy": 10951918, "alias": "Doran", "name": "DoranBell", "user-since": datetime("2005-08-22T14:07:50.000Z"), "user-since-copy": datetime("2005-08-22T14:07:50.000Z"), "friend-ids": {{ 6952033, 22223086, 5858716, 35128893, 22115927, 5821006, 16264772, 4151991, 40384467, 19801357, 42871024, 46855275, 35241988, 17208259, 47420533, 25182232, 14247140, 19664015, 33132502, 47813026, 12819081, 29321093, 42851957, 30756972 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2012-01-22"), "end-date": null } ] }
+{ "id": 10962466, "id-copy": 10962466, "alias": "Zoey", "name": "ZoeyCady", "user-since": datetime("2012-07-15T20:02:23.000Z"), "user-since-copy": datetime("2012-07-15T20:02:23.000Z"), "friend-ids": {{ 12726157, 268799, 29381478, 15699674, 1150948, 8000369, 41608951, 11382366, 770690, 25889785, 37815043, 40437016, 38679636, 32956275, 34853801 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2012-05-08"), "end-date": null } ] }
+{ "id": 10968562, "id-copy": 10968562, "alias": "Fox", "name": "FoxBillimek", "user-since": datetime("2012-03-24T07:32:17.000Z"), "user-since-copy": datetime("2012-03-24T07:32:17.000Z"), "friend-ids": {{ 8459327, 11505750, 30952882, 30467951, 6329439, 33947538, 19579432, 25135787, 41391398, 32456626, 6310287, 31211659 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2009-01-17"), "end-date": null } ] }
+{ "id": 10970950, "id-copy": 10970950, "alias": "Shana", "name": "ShanaRose", "user-since": datetime("2008-09-17T10:03:01.000Z"), "user-since-copy": datetime("2008-09-17T10:03:01.000Z"), "friend-ids": {{ 21025589, 17977659, 39920039, 44311386, 2634251 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2006-09-17"), "end-date": null } ] }
+{ "id": 11003527, "id-copy": 11003527, "alias": "Clitus", "name": "ClitusDickinson", "user-since": datetime("2007-10-18T04:59:18.000Z"), "user-since-copy": datetime("2007-10-18T04:59:18.000Z"), "friend-ids": {{ 26264340, 47892511, 18715043, 43994375, 42874707, 44696774, 7281939 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2006-09-15"), "end-date": null } ] }
+{ "id": 11004067, "id-copy": 11004067, "alias": "Vickie", "name": "VickieRosenstiehl", "user-since": datetime("2012-04-15T02:37:43.000Z"), "user-since-copy": datetime("2012-04-15T02:37:43.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2012-08-01"), "end-date": date("2012-08-06") } ] }
+{ "id": 11010904, "id-copy": 11010904, "alias": "Chang", "name": "ChangSteele", "user-since": datetime("2009-02-24T01:43:56.000Z"), "user-since-copy": datetime("2009-02-24T01:43:56.000Z"), "friend-ids": {{ 19212881, 4019921, 24976558, 47613555, 26049623, 17656988, 24011085, 31763054, 21741933, 31356824, 9651386, 35034682, 5665574, 31306405, 38922156, 9837341, 31865250, 12415354 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2005-09-20"), "end-date": date("2005-05-28") } ] }
+{ "id": 11022826, "id-copy": 11022826, "alias": "Virgee", "name": "VirgeeHolts", "user-since": datetime("2012-01-17T22:54:54.000Z"), "user-since-copy": datetime("2012-01-17T22:54:54.000Z"), "friend-ids": {{ 40134062, 13624785, 23477090, 26708578, 18967215, 21325604, 15522457, 25873528 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2004-05-09"), "end-date": date("2010-06-15") } ] }
+{ "id": 11039716, "id-copy": 11039716, "alias": "Piedad", "name": "PiedadHowe", "user-since": datetime("2011-02-23T17:18:37.000Z"), "user-since-copy": datetime("2011-02-23T17:18:37.000Z"), "friend-ids": {{ 13323345 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2009-03-26"), "end-date": date("2009-06-17") } ] }
+{ "id": 11049274, "id-copy": 11049274, "alias": "Fitz", "name": "FitzBeail", "user-since": datetime("2012-08-10T03:25:57.000Z"), "user-since-copy": datetime("2012-08-10T03:25:57.000Z"), "friend-ids": {{ 39403330, 13441324, 723509, 34025727, 23266816, 33898717, 11053310, 14582395, 38435153, 45855468, 45712821 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2004-02-16"), "end-date": date("2007-01-07") } ] }
+{ "id": 11059435, "id-copy": 11059435, "alias": "Lucina", "name": "LucinaDurstine", "user-since": datetime("2007-04-14T19:19:23.000Z"), "user-since-copy": datetime("2007-04-14T19:19:23.000Z"), "friend-ids": {{ 18983436, 36225185, 42601602, 22134709, 20671612 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2009-06-15"), "end-date": null } ] }
+{ "id": 11062330, "id-copy": 11062330, "alias": "Derick", "name": "DerickPennington", "user-since": datetime("2008-04-15T11:59:52.000Z"), "user-since-copy": datetime("2008-04-15T11:59:52.000Z"), "friend-ids": {{ 26471368, 22445928, 13709179, 16677606, 45234923, 5601330, 16510085, 27673980, 24365707, 42647605, 20473849, 40448252, 37480913, 38532114, 11022656, 799537, 38469920, 1291033, 31503804, 29154535, 5506108, 24609403, 35535409, 44197253 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2002-09-23"), "end-date": null } ] }
+{ "id": 11068231, "id-copy": 11068231, "alias": "Dinah", "name": "DinahSwink", "user-since": datetime("2012-05-02T04:24:33.000Z"), "user-since-copy": datetime("2012-05-02T04:24:33.000Z"), "friend-ids": {{ 31542440, 17451543, 32642661, 27867264, 32718667, 43042567, 7921827 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2003-04-10"), "end-date": date("2003-10-03") } ] }
+{ "id": 11087839, "id-copy": 11087839, "alias": "Manfred", "name": "ManfredEdwards", "user-since": datetime("2009-10-01T09:12:15.000Z"), "user-since-copy": datetime("2009-10-01T09:12:15.000Z"), "friend-ids": {{ 7828089 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2003-07-25"), "end-date": null } ] }
+{ "id": 11090788, "id-copy": 11090788, "alias": "Randy", "name": "RandyClose", "user-since": datetime("2005-07-26T19:29:20.000Z"), "user-since-copy": datetime("2005-07-26T19:29:20.000Z"), "friend-ids": {{ 43392502, 7581874, 13279708, 16989391, 32340594, 7048512, 33084049, 16279611, 21735714, 23485799, 18185370, 43945382, 41653020, 13517043, 35395274, 24133848, 15355027, 4752815, 15007500, 25733540, 2114558, 37909789, 2805493, 16521087 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2004-09-14"), "end-date": null } ] }
+{ "id": 11097556, "id-copy": 11097556, "alias": "Tia", "name": "TiaHair", "user-since": datetime("2010-10-28T01:21:36.000Z"), "user-since-copy": datetime("2010-10-28T01:21:36.000Z"), "friend-ids": {{ 19746022, 42650092, 45679457, 43873545, 5490025, 42900988, 32855768, 20717716, 15007194, 23035301, 24322095, 27796211, 27751858, 4726224, 5570083, 18421959, 28424121, 22311092, 13781420, 18215783, 19934706, 18408890, 24792739, 4022527 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2003-04-03"), "end-date": null } ] }
+{ "id": 11109553, "id-copy": 11109553, "alias": "Walker", "name": "WalkerDrennan", "user-since": datetime("2007-05-03T02:10:46.000Z"), "user-since-copy": datetime("2007-05-03T02:10:46.000Z"), "friend-ids": {{ 38288636, 35385410, 24803705, 31461936, 34309407 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2010-05-20"), "end-date": null } ] }
+{ "id": 11131756, "id-copy": 11131756, "alias": "Sharlene", "name": "SharleneFinlay", "user-since": datetime("2006-01-11T00:34:50.000Z"), "user-since-copy": datetime("2006-01-11T00:34:50.000Z"), "friend-ids": {{ 47024803, 17225785, 29871165, 14503159, 22992924, 38939801, 44563447, 101625, 40957129, 24838380, 7187619, 45283524, 31617405, 517806, 28714183, 32966332, 24006006 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2008-02-16"), "end-date": date("2011-09-12") } ] }
+{ "id": 11147392, "id-copy": 11147392, "alias": "Sarina", "name": "SarinaFlickinger", "user-since": datetime("2011-09-26T12:41:56.000Z"), "user-since-copy": datetime("2011-09-26T12:41:56.000Z"), "friend-ids": {{ 17776087, 9254087, 14735666, 31097664, 36421253, 12595115, 40366588, 9491701, 29725314, 38852857, 46206259, 39281843, 36268114, 29939350, 804107, 36307361, 30999436, 47369074, 3820973, 46362092, 36413930, 8807546, 30260636, 15069463 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2005-03-15"), "end-date": null } ] }
+{ "id": 11155816, "id-copy": 11155816, "alias": "Titty", "name": "TittyOneal", "user-since": datetime("2009-06-01T06:21:44.000Z"), "user-since-copy": datetime("2009-06-01T06:21:44.000Z"), "friend-ids": {{ 37016026, 32220220, 47720886, 10358045, 7678433, 22148913, 18800507, 17043803, 29852152, 11426875, 44761613, 32002053, 14686180, 26744098, 34991446, 38818677, 24977770 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2012-05-11"), "end-date": date("2012-05-08") } ] }
+{ "id": 11179192, "id-copy": 11179192, "alias": "Derren", "name": "DerrenClose", "user-since": datetime("2008-04-28T09:18:19.000Z"), "user-since-copy": datetime("2008-04-28T09:18:19.000Z"), "friend-ids": {{ 43947479, 30154889, 10673575, 8056171, 28691242, 22881730, 15291446, 7331632, 32819016, 35194153 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-10-24"), "end-date": date("2006-08-12") } ] }
+{ "id": 11195221, "id-copy": 11195221, "alias": "Clement", "name": "ClementBriner", "user-since": datetime("2006-12-27T02:29:02.000Z"), "user-since-copy": datetime("2006-12-27T02:29:02.000Z"), "friend-ids": {{ 33023290 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2000-06-05"), "end-date": null } ] }
+{ "id": 11216260, "id-copy": 11216260, "alias": "Randy", "name": "RandyEckhardstein", "user-since": datetime("2006-12-05T07:09:34.000Z"), "user-since-copy": datetime("2006-12-05T07:09:34.000Z"), "friend-ids": {{ 39744737, 14315897, 1342674, 1761832, 41393930, 21351330, 17845632, 39034426, 15297881, 11656496, 11376855 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2009-12-19"), "end-date": null } ] }
+{ "id": 11233525, "id-copy": 11233525, "alias": "Syd", "name": "SydSauter", "user-since": datetime("2010-12-18T02:44:55.000Z"), "user-since-copy": datetime("2010-12-18T02:44:55.000Z"), "friend-ids": {{ 6312313, 17431246, 36729581, 3715101, 39534341, 10333995, 36042764, 14014852, 27375328, 17089631, 24066240, 42616402, 34049424, 29807262, 25669160, 43435752, 46702290, 27418631, 13587383, 14811241 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2010-06-21"), "end-date": null } ] }
+{ "id": 11244439, "id-copy": 11244439, "alias": "Francene", "name": "FranceneArmstrong", "user-since": datetime("2009-11-12T19:32:27.000Z"), "user-since-copy": datetime("2009-11-12T19:32:27.000Z"), "friend-ids": {{ 27784445, 37528954, 14014093, 18695376 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2000-06-26"), "end-date": null } ] }
+{ "id": 11250445, "id-copy": 11250445, "alias": "Charlie", "name": "CharlieHaynes", "user-since": datetime("2009-06-08T22:50:05.000Z"), "user-since-copy": datetime("2009-06-08T22:50:05.000Z"), "friend-ids": {{ 18548568, 33185990, 25924893, 44738376, 17285644, 30895698, 40664753, 45663520, 13757940, 46543434, 27472319, 7112791, 45257808, 29363383, 24726693, 39990597, 36277676, 6623887, 42795972, 29019649, 22035134, 1362080, 9071131 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2008-02-21"), "end-date": date("2009-12-28") } ] }
+{ "id": 11252185, "id-copy": 11252185, "alias": "Quintin", "name": "QuintinMcdonald", "user-since": datetime("2010-09-27T08:09:51.000Z"), "user-since-copy": datetime("2010-09-27T08:09:51.000Z"), "friend-ids": {{ 17231767, 1840658, 32389773, 31328720, 18446903, 48007173, 40417004, 41543048, 4774035, 43047815, 24232919, 936390, 20744224, 39536211, 34205950, 38429209, 399190, 38425767, 8776604, 10360244, 28414116, 15735235, 6431904 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-10-04"), "end-date": null } ] }
+{ "id": 11280553, "id-copy": 11280553, "alias": "Wendy", "name": "WendyClarke", "user-since": datetime("2009-08-28T16:53:37.000Z"), "user-since-copy": datetime("2009-08-28T16:53:37.000Z"), "friend-ids": {{ 10802559, 42649709, 8824750, 19241403, 43339000, 23865070, 9842110, 7051904, 39440876, 16961992 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2004-11-15"), "end-date": date("2005-01-15") } ] }
+{ "id": 11290870, "id-copy": 11290870, "alias": "Lanford", "name": "LanfordOsteen", "user-since": datetime("2009-03-04T15:04:12.000Z"), "user-since-copy": datetime("2009-03-04T15:04:12.000Z"), "friend-ids": {{ 4397941, 36140649, 12796618, 18235191, 8810154, 10521988, 6580979, 29578654, 46083953, 30113784, 25952539 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2009-08-06"), "end-date": null } ] }
+{ "id": 11306677, "id-copy": 11306677, "alias": "Chong", "name": "ChongPawle", "user-since": datetime("2007-09-13T00:31:41.000Z"), "user-since-copy": datetime("2007-09-13T00:31:41.000Z"), "friend-ids": {{ 11341417, 23669364, 41504484, 29889550, 268223, 26888454, 43915376, 23795433, 14021648, 25630355, 19831181, 15828987 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2011-01-06"), "end-date": date("2011-10-06") } ] }
+{ "id": 11307037, "id-copy": 11307037, "alias": "Brett", "name": "BrettLeichter", "user-since": datetime("2011-02-24T01:38:23.000Z"), "user-since-copy": datetime("2011-02-24T01:38:23.000Z"), "friend-ids": {{ 16273758, 36959770, 26721660 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2011-11-23"), "end-date": null } ] }
+{ "id": 11307946, "id-copy": 11307946, "alias": "Helga", "name": "HelgaStough", "user-since": datetime("2007-01-12T21:50:11.000Z"), "user-since-copy": datetime("2007-01-12T21:50:11.000Z"), "friend-ids": {{ 22768365 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2007-01-04"), "end-date": date("2009-06-25") } ] }
+{ "id": 11316178, "id-copy": 11316178, "alias": "Carlene", "name": "CarleneArchibald", "user-since": datetime("2007-09-02T16:24:57.000Z"), "user-since-copy": datetime("2007-09-02T16:24:57.000Z"), "friend-ids": {{ 45522809, 33213012, 2265630, 27087141, 7247502, 38659338, 33327692, 43927391, 41809132, 4738869, 9663680, 45809341, 38204579, 17145650, 23991333, 9915598, 28129675, 47406993, 37554697 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2007-12-15"), "end-date": date("2008-06-02") } ] }
+{ "id": 11318098, "id-copy": 11318098, "alias": "Lucilla", "name": "LucillaSteele", "user-since": datetime("2006-05-02T12:10:51.000Z"), "user-since-copy": datetime("2006-05-02T12:10:51.000Z"), "friend-ids": {{ 43202249, 11116520, 19404968, 23494384, 41664359, 2459832, 21895811, 29849475, 32963400, 24381723, 46790616, 10343240, 43849340, 16769526, 26104853 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2009-10-09"), "end-date": null } ] }
+{ "id": 11335972, "id-copy": 11335972, "alias": "Emmett", "name": "EmmettBaxter", "user-since": datetime("2008-04-25T01:22:30.000Z"), "user-since-copy": datetime("2008-04-25T01:22:30.000Z"), "friend-ids": {{ 23133373, 28796661, 13045317, 34201656, 44749284, 42654826, 988887, 5039257, 18280226, 30366668, 22387991, 32676638, 24149069, 6307083, 17556069, 16687473, 4101198, 41964241, 39245728 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2004-11-22"), "end-date": null } ] }
+{ "id": 11357614, "id-copy": 11357614, "alias": "Denys", "name": "DenysMcintosh", "user-since": datetime("2006-01-15T22:32:48.000Z"), "user-since-copy": datetime("2006-01-15T22:32:48.000Z"), "friend-ids": {{ 10713170, 21699820, 14949046, 7935772, 21404351, 21078565, 15867691, 41676271, 2655928, 22987809, 16585582, 8318693, 46886662, 15081903, 47617713, 6317213, 32997127 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2008-08-28"), "end-date": null } ] }
+{ "id": 11362531, "id-copy": 11362531, "alias": "Garey", "name": "GareyChapman", "user-since": datetime("2005-10-13T04:24:29.000Z"), "user-since-copy": datetime("2005-10-13T04:24:29.000Z"), "friend-ids": {{ 20693565, 18896854, 17118168, 12285534, 21434048, 15453439, 42734432, 3627967, 30464042, 11556192, 22808282, 464074, 28100870, 29887664, 19046987, 34996619, 39964690, 22574200, 29497238 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2001-03-05"), "end-date": null } ] }
+{ "id": 11373598, "id-copy": 11373598, "alias": "Dina", "name": "DinaDriggers", "user-since": datetime("2010-01-06T22:56:18.000Z"), "user-since-copy": datetime("2010-01-06T22:56:18.000Z"), "friend-ids": {{ 8839886, 10146989, 10877857, 11710726, 5699142, 27984085, 12834284 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2012-07-25"), "end-date": null } ] }
+{ "id": 11381089, "id-copy": 11381089, "alias": "Earlene", "name": "EarleneAmmons", "user-since": datetime("2010-03-24T05:25:35.000Z"), "user-since-copy": datetime("2010-03-24T05:25:35.000Z"), "friend-ids": {{ 25392364, 36996951, 16110083, 9799716, 22893553, 28551996, 7706432, 14225386, 15633254, 39395931, 46707062, 37226919, 8532306, 3765988, 20939685, 31136325, 45222021, 15355741, 8760941, 12045616, 6890610, 13560532, 44914868, 37744233 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2000-06-10"), "end-date": null } ] }
+{ "id": 11390830, "id-copy": 11390830, "alias": "Luciano", "name": "LucianoHooker", "user-since": datetime("2006-08-16T08:17:56.000Z"), "user-since-copy": datetime("2006-08-16T08:17:56.000Z"), "friend-ids": {{ 42206490, 5533465, 32480435, 18058343 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2004-02-19"), "end-date": null } ] }
+{ "id": 11403742, "id-copy": 11403742, "alias": "Neil", "name": "NeilHobbs", "user-since": datetime("2012-02-26T07:07:17.000Z"), "user-since-copy": datetime("2012-02-26T07:07:17.000Z"), "friend-ids": {{ 28387528, 39844931, 32868894, 45540524, 35239986, 44255870, 20859099 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2008-11-28"), "end-date": date("2009-06-01") } ] }
+{ "id": 11416066, "id-copy": 11416066, "alias": "Janna", "name": "JannaBowchiew", "user-since": datetime("2010-12-06T10:53:56.000Z"), "user-since-copy": datetime("2010-12-06T10:53:56.000Z"), "friend-ids": {{ 43816151, 22032304, 27239988, 23813127, 34936097, 8817657, 39872787, 27628236, 38333824, 40879066 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2001-04-19"), "end-date": date("2008-01-09") } ] }
+{ "id": 11424097, "id-copy": 11424097, "alias": "Vernie", "name": "VernieWynter", "user-since": datetime("2009-02-15T02:35:16.000Z"), "user-since-copy": datetime("2009-02-15T02:35:16.000Z"), "friend-ids": {{ 41874621, 26330221, 38930134, 39892396, 42859035, 8165423, 36128938, 5692990, 28144348, 40741492 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2002-04-06"), "end-date": null } ] }
+{ "id": 11425216, "id-copy": 11425216, "alias": "Levi", "name": "LeviEiford", "user-since": datetime("2010-04-10T23:37:26.000Z"), "user-since-copy": datetime("2010-04-10T23:37:26.000Z"), "friend-ids": {{ 39348801, 15029457, 33995161, 27782571, 16712478, 28987111 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2002-08-12"), "end-date": null } ] }
+{ "id": 11435779, "id-copy": 11435779, "alias": "Jonty", "name": "JontyLarson", "user-since": datetime("2012-04-11T08:34:47.000Z"), "user-since-copy": datetime("2012-04-11T08:34:47.000Z"), "friend-ids": {{ 37343432, 9979565, 14647518, 32490112, 26673699, 22447290, 40923710, 47426439 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2010-08-24"), "end-date": date("2011-06-21") } ] }
+{ "id": 11437771, "id-copy": 11437771, "alias": "Brittani", "name": "BrittaniMoore", "user-since": datetime("2007-11-16T20:56:35.000Z"), "user-since-copy": datetime("2007-11-16T20:56:35.000Z"), "friend-ids": {{ 30502334, 18483492, 37360877, 25153720, 9181228, 28352241, 37928337, 13522608, 20974146, 30187156, 22832401, 20899789, 44606652, 3333090, 39581573, 34303132, 33802071, 27053375, 32467186, 40213342, 37254307, 7275338, 2622767 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2010-02-07"), "end-date": null } ] }
+{ "id": 11445889, "id-copy": 11445889, "alias": "Milford", "name": "MilfordTeagarden", "user-since": datetime("2006-06-07T19:18:28.000Z"), "user-since-copy": datetime("2006-06-07T19:18:28.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "subtam", "start-date": date("2003-07-26"), "end-date": null } ] }
+{ "id": 11454253, "id-copy": 11454253, "alias": "Fairy", "name": "FairyFoster", "user-since": datetime("2007-05-04T11:48:12.000Z"), "user-since-copy": datetime("2007-05-04T11:48:12.000Z"), "friend-ids": {{ 15077027, 13719617, 3663639, 16159577, 29937764, 11018999, 36883485, 35967804, 16558412, 19456409, 33156277, 8763694, 9279896 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2012-07-10"), "end-date": null } ] }
+{ "id": 11455492, "id-copy": 11455492, "alias": "Cymbeline", "name": "CymbelineEliza", "user-since": datetime("2010-05-03T21:32:10.000Z"), "user-since-copy": datetime("2010-05-03T21:32:10.000Z"), "friend-ids": {{ 27738860, 21711920, 47805508, 33507501, 22648267, 1006513, 23617648, 20104970, 8132761, 14963107, 19477123 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2002-07-25"), "end-date": null } ] }
+{ "id": 11463820, "id-copy": 11463820, "alias": "Gaye", "name": "GayeWelty", "user-since": datetime("2005-01-04T14:32:34.000Z"), "user-since-copy": datetime("2005-01-04T14:32:34.000Z"), "friend-ids": {{ 44428980, 1291384, 10830264, 2433795, 17582948, 17416624, 21578025, 14538036, 41470487, 34384402, 42863727, 35119046, 35673193, 14814350, 29380258, 30253821, 41180218, 13945680, 15533641, 26510747 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2011-03-01"), "end-date": date("2011-09-13") } ] }
+{ "id": 11474374, "id-copy": 11474374, "alias": "Waldo", "name": "WaldoKnapp", "user-since": datetime("2008-08-17T21:17:28.000Z"), "user-since-copy": datetime("2008-08-17T21:17:28.000Z"), "friend-ids": {{ 33358772, 16499546, 8631001, 6045567, 45554236, 36229482, 354579, 11884970, 23657774, 32568373 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2004-11-18"), "end-date": null } ] }
+{ "id": 11489143, "id-copy": 11489143, "alias": "Clover", "name": "CloverWest", "user-since": datetime("2012-04-14T13:56:22.000Z"), "user-since-copy": datetime("2012-04-14T13:56:22.000Z"), "friend-ids": {{ 14606516, 25835971, 10555192, 4853088, 43631398, 45670230, 43866490, 25690294, 22040370, 7047997, 3374421, 34831455, 31517002, 2998558, 40893307, 40067725, 1601716, 43041725, 8953042, 33848939 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2005-02-03"), "end-date": date("2006-06-26") } ] }
+{ "id": 11490220, "id-copy": 11490220, "alias": "Ernestine", "name": "ErnestineWheeler", "user-since": datetime("2005-01-27T23:36:35.000Z"), "user-since-copy": datetime("2005-01-27T23:36:35.000Z"), "friend-ids": {{ 12995063, 40353122, 11162426, 42762839, 9575788, 7725738, 29883894, 48002015, 5516807, 12731814, 33203496, 44912740, 19681146, 5849671, 4702317 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2008-06-16"), "end-date": date("2011-12-01") } ] }
+{ "id": 11507149, "id-copy": 11507149, "alias": "Kendal", "name": "KendalCourtney", "user-since": datetime("2006-06-22T04:28:09.000Z"), "user-since-copy": datetime("2006-06-22T04:28:09.000Z"), "friend-ids": {{ 9084267, 26163683, 15271756, 4229254, 5439809, 23992890, 23144677, 26584955, 29430424, 15196312, 19993838, 3665259, 15861241, 15197583, 15693177 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2010-08-06"), "end-date": date("2011-04-21") } ] }
+{ "id": 11518480, "id-copy": 11518480, "alias": "Amada", "name": "AmadaTanner", "user-since": datetime("2006-05-06T12:27:31.000Z"), "user-since-copy": datetime("2006-05-06T12:27:31.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2002-04-02"), "end-date": null } ] }
+{ "id": 11529730, "id-copy": 11529730, "alias": "Linwood", "name": "LinwoodZadovsky", "user-since": datetime("2007-03-13T03:41:20.000Z"), "user-since-copy": datetime("2007-03-13T03:41:20.000Z"), "friend-ids": {{ 23516069, 24312236, 23750591, 36982495, 36483830 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2008-01-25"), "end-date": null } ] }
+{ "id": 11538001, "id-copy": 11538001, "alias": "Milo", "name": "MiloGarland", "user-since": datetime("2007-09-12T09:40:42.000Z"), "user-since-copy": datetime("2007-09-12T09:40:42.000Z"), "friend-ids": {{ 7363153, 7252759 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2011-09-03"), "end-date": date("2011-10-27") } ] }
+{ "id": 11551078, "id-copy": 11551078, "alias": "Percy", "name": "PercyStocker", "user-since": datetime("2012-01-12T15:14:02.000Z"), "user-since-copy": datetime("2012-01-12T15:14:02.000Z"), "friend-ids": {{ 8927010, 25565873, 1309019, 9736505, 27953053, 6619625, 45562540, 32022492, 1535156, 11343220, 40057278, 5452463, 36005348, 35072612, 31954888 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2004-06-01"), "end-date": date("2010-03-09") } ] }
+{ "id": 11559262, "id-copy": 11559262, "alias": "Herb", "name": "HerbPaul", "user-since": datetime("2011-04-09T22:23:26.000Z"), "user-since-copy": datetime("2011-04-09T22:23:26.000Z"), "friend-ids": {{ 46915837, 26659094 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2011-07-05"), "end-date": date("2011-07-07") } ] }
+{ "id": 11582299, "id-copy": 11582299, "alias": "Seward", "name": "SewardReddish", "user-since": datetime("2007-11-07T11:10:00.000Z"), "user-since-copy": datetime("2007-11-07T11:10:00.000Z"), "friend-ids": {{ 14793773, 24447668, 30727802, 4757816, 26139324, 4433524, 15974482 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2012-02-10"), "end-date": null } ] }
+{ "id": 11587666, "id-copy": 11587666, "alias": "Kathi", "name": "KathiJenner", "user-since": datetime("2012-02-20T01:58:30.000Z"), "user-since-copy": datetime("2012-02-20T01:58:30.000Z"), "friend-ids": {{ 37156773, 10519382, 11009989, 47883115, 13123467, 36990044, 8554049, 47075065, 11896169, 42580126, 43261036, 15337748, 35985068, 44438965, 33507413, 40063633, 32559158, 32202309, 25536635 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2001-01-01"), "end-date": null } ] }
+{ "id": 11588467, "id-copy": 11588467, "alias": "Soon", "name": "SoonHays", "user-since": datetime("2011-12-21T05:33:54.000Z"), "user-since-copy": datetime("2011-12-21T05:33:54.000Z"), "friend-ids": {{ 659930 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2009-05-20"), "end-date": date("2009-07-16") } ] }
+{ "id": 11596522, "id-copy": 11596522, "alias": "Gena", "name": "GenaTurzanski", "user-since": datetime("2012-06-22T18:42:25.000Z"), "user-since-copy": datetime("2012-06-22T18:42:25.000Z"), "friend-ids": {{ 22525625, 22327219, 18520174, 38679685, 16561552, 1999972, 8066310, 24245231, 11682156, 31330371, 38780021, 46833789, 6710024, 38963740, 38984150, 33451484, 19022059, 36880540, 40003274 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2002-09-10"), "end-date": null } ] }
+{ "id": 11617963, "id-copy": 11617963, "alias": "Sherry", "name": "SherryPirl", "user-since": datetime("2010-08-26T06:37:30.000Z"), "user-since-copy": datetime("2010-08-26T06:37:30.000Z"), "friend-ids": {{ 30179664, 7140787, 14622079, 5810238, 32189583, 17103583 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2000-02-07"), "end-date": date("2004-11-24") } ] }
+{ "id": 11626990, "id-copy": 11626990, "alias": "Filiberto", "name": "FilibertoFonblanque", "user-since": datetime("2006-05-18T07:38:32.000Z"), "user-since-copy": datetime("2006-05-18T07:38:32.000Z"), "friend-ids": {{ 41443868, 30006940, 14137070, 14868792, 47991977, 39513958, 32787637, 1389727, 28607710, 21537795, 42395037, 11730902, 25246772, 24475669, 35786951, 32795214 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2007-08-11"), "end-date": null } ] }
+{ "id": 11637820, "id-copy": 11637820, "alias": "Aislin", "name": "AislinPyle", "user-since": datetime("2005-01-04T00:11:51.000Z"), "user-since-copy": datetime("2005-01-04T00:11:51.000Z"), "friend-ids": {{ 17232277, 46376966, 22503632, 14771156, 37550654, 3930020, 7116826, 38303815, 30210948, 10532544, 44382464, 32051602 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2004-05-06"), "end-date": null } ] }
+{ "id": 11659888, "id-copy": 11659888, "alias": "Nannie", "name": "NannieWoodworth", "user-since": datetime("2006-12-11T15:30:08.000Z"), "user-since-copy": datetime("2006-12-11T15:30:08.000Z"), "friend-ids": {{ 30803046, 33105462, 14783423, 5069473, 15960335 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2006-10-12"), "end-date": null } ] }
+{ "id": 11668552, "id-copy": 11668552, "alias": "Kassandra", "name": "KassandraJames", "user-since": datetime("2010-09-27T18:12:59.000Z"), "user-since-copy": datetime("2010-09-27T18:12:59.000Z"), "friend-ids": {{ 27400643, 15449089, 802964, 45059523, 9603951, 20911122, 46243977, 45487995, 34528880, 16093159, 22484957, 3951663, 12349433, 7887502, 34786818, 13014384, 28307526, 30476565, 7746152, 17600641, 36877141, 4513081, 25065078 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2012-08-04"), "end-date": date("2012-08-25") } ] }
+{ "id": 11670331, "id-copy": 11670331, "alias": "Deetta", "name": "DeettaCrom", "user-since": datetime("2008-04-01T00:12:47.000Z"), "user-since-copy": datetime("2008-04-01T00:12:47.000Z"), "friend-ids": {{ 34871046, 45366633, 40484162, 45505621, 47279131, 5464046, 18435436, 24937987, 18253019, 5870229, 46379232, 13988659, 37921800, 2085103, 21652843, 4802881, 11658526, 40771399, 32938488, 8409007, 27179341, 4496744 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2003-10-21"), "end-date": date("2008-06-06") } ] }
+{ "id": 11678242, "id-copy": 11678242, "alias": "Andy", "name": "AndyPritchard", "user-since": datetime("2008-05-26T06:52:12.000Z"), "user-since-copy": datetime("2008-05-26T06:52:12.000Z"), "friend-ids": {{ 24351029, 7396495, 11653891, 24314059, 17256129, 19177689, 23024021, 15135862, 9201238, 24204194 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2004-02-06"), "end-date": date("2011-10-22") } ] }
+{ "id": 11721010, "id-copy": 11721010, "alias": "Eliot", "name": "EliotTennant", "user-since": datetime("2009-07-25T22:16:20.000Z"), "user-since-copy": datetime("2009-07-25T22:16:20.000Z"), "friend-ids": {{ 41972338, 13293762, 47012929, 13695904, 25235210, 39246961, 36832468, 26854695, 3046764, 17117110, 10902219, 36959080, 32665222 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-11-26"), "end-date": null } ] }
+{ "id": 11723506, "id-copy": 11723506, "alias": "Odelia", "name": "OdeliaPaul", "user-since": datetime("2006-03-14T15:49:03.000Z"), "user-since-copy": datetime("2006-03-14T15:49:03.000Z"), "friend-ids": {{ 874326, 37021972, 27293893, 40453006, 44728117, 338941, 22832206, 11391929, 46420525 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2012-05-05"), "end-date": null } ] }
+{ "id": 11748019, "id-copy": 11748019, "alias": "Malinda", "name": "MalindaMoberly", "user-since": datetime("2005-06-21T22:34:38.000Z"), "user-since-copy": datetime("2005-06-21T22:34:38.000Z"), "friend-ids": {{ 46792750, 47197275, 45940765, 43931611, 33201251, 32508732, 23681521, 35069089, 43652710, 22676488, 5098654, 29592897, 18671070, 40200423 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2004-08-18"), "end-date": null } ] }
+{ "id": 11755633, "id-copy": 11755633, "alias": "Amina", "name": "AminaBurkett", "user-since": datetime("2012-03-22T02:05:59.000Z"), "user-since-copy": datetime("2012-03-22T02:05:59.000Z"), "friend-ids": {{ 18177270, 40223354, 29458819, 37905784, 43047863, 2679271, 9768971, 32443429, 37829920, 35493852, 28086857, 11910843, 31003179, 40873211, 42786132, 44388462 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2009-11-21"), "end-date": date("2011-03-16") } ] }
+{ "id": 11781745, "id-copy": 11781745, "alias": "Merv", "name": "MervStocker", "user-since": datetime("2008-10-15T03:41:54.000Z"), "user-since-copy": datetime("2008-10-15T03:41:54.000Z"), "friend-ids": {{ 26394519, 2599602, 40237077, 43817129, 30392481, 43051494, 36128635, 35974184, 37237292, 7775912, 11569464, 9112021, 26837692, 11548106, 29331601, 11126182, 18076463, 33866145, 22408972, 42318835, 47199541, 26807788 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2005-01-15"), "end-date": date("2008-02-18") } ] }
+{ "id": 11782354, "id-copy": 11782354, "alias": "Glynda", "name": "GlyndaEnderly", "user-since": datetime("2007-11-25T06:01:45.000Z"), "user-since-copy": datetime("2007-11-25T06:01:45.000Z"), "friend-ids": {{ 16202981, 24035766, 10175614, 27353200, 26183740, 6084065, 31664832, 22446721, 2792685, 37521374, 1999182, 12494503, 18087992, 44433851 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2004-06-10"), "end-date": null } ] }
+{ "id": 11786815, "id-copy": 11786815, "alias": "Micheal", "name": "MichealTreeby", "user-since": datetime("2008-06-04T14:59:23.000Z"), "user-since-copy": datetime("2008-06-04T14:59:23.000Z"), "friend-ids": {{ 15590922, 1367468, 37464776, 21877607, 38646966, 46702919, 46771039, 4688915, 41827211, 6556380 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2003-09-17"), "end-date": null } ] }
+{ "id": 11788345, "id-copy": 11788345, "alias": "Mindy", "name": "MindyRockwell", "user-since": datetime("2011-02-20T23:55:16.000Z"), "user-since-copy": datetime("2011-02-20T23:55:16.000Z"), "friend-ids": {{ 7821092, 24614722, 27718237, 19686343, 43916267, 7882804, 34422272, 46273261, 658009, 42620170, 36177155, 3340224, 27157340, 20438623, 19694381, 15643415, 43465380, 17719224, 37073374, 42060457, 29532671, 3781069, 26121650 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2011-05-11"), "end-date": null } ] }
+{ "id": 11788834, "id-copy": 11788834, "alias": "Benny", "name": "BennyAgg", "user-since": datetime("2011-12-19T14:28:16.000Z"), "user-since-copy": datetime("2011-12-19T14:28:16.000Z"), "friend-ids": {{ 6023130, 41817759, 15338300, 40598251, 38750529, 43646078, 9057658 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2006-09-16"), "end-date": null } ] }
+{ "id": 11821996, "id-copy": 11821996, "alias": "Latanya", "name": "LatanyaZalack", "user-since": datetime("2010-12-07T15:20:09.000Z"), "user-since-copy": datetime("2010-12-07T15:20:09.000Z"), "friend-ids": {{ 23521495, 43957220, 3823403, 34033770 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2008-04-17"), "end-date": null } ] }
+{ "id": 11840218, "id-copy": 11840218, "alias": "Deandre", "name": "DeandreMackendrick", "user-since": datetime("2012-07-03T08:22:13.000Z"), "user-since-copy": datetime("2012-07-03T08:22:13.000Z"), "friend-ids": {{ 36310775, 13455844, 1133499, 44183463, 28002311, 40758157, 33299342, 47526543, 9613784, 5698202, 1492720, 5663846 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2006-03-12"), "end-date": date("2009-08-08") } ] }
+{ "id": 11857618, "id-copy": 11857618, "alias": "Glenda", "name": "GlendaPyle", "user-since": datetime("2009-01-05T13:34:53.000Z"), "user-since-copy": datetime("2009-01-05T13:34:53.000Z"), "friend-ids": {{ 31083833, 39371819, 38336556, 7590988, 17022330, 8016611, 41444367, 13194826, 1589028, 37076285, 33481940, 22093098, 9959371, 35262849, 20744580, 33226729, 35025566, 46396680, 30247311, 6884899, 35691024, 40965552, 46106170 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2000-02-19"), "end-date": null } ] }
+{ "id": 11878948, "id-copy": 11878948, "alias": "Corey", "name": "CoreyWarrick", "user-since": datetime("2005-05-28T15:18:23.000Z"), "user-since-copy": datetime("2005-05-28T15:18:23.000Z"), "friend-ids": {{ 17192577, 19646534, 44755348, 28653064, 30539369, 15001411, 11921646, 44450607, 33599896, 41984600, 2187246, 8785209, 28099595 }}, "employment": [ { "organization-name": "Zimcone", "start-date": date("2010-12-07"), "end-date": null } ] }
+{ "id": 11886709, "id-copy": 11886709, "alias": "Leigh", "name": "LeighBatten", "user-since": datetime("2005-06-18T21:25:13.000Z"), "user-since-copy": datetime("2005-06-18T21:25:13.000Z"), "friend-ids": {{ 161610, 3498914, 24173074, 33102324, 42213688, 44551300, 36373040, 30704767, 24224319, 5784194, 13092764, 38315503, 13246046, 2836280, 672136, 37021775 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2001-05-26"), "end-date": date("2001-05-11") } ] }
+{ "id": 11894854, "id-copy": 11894854, "alias": "Connor", "name": "ConnorWilliamson", "user-since": datetime("2011-09-16T22:24:17.000Z"), "user-since-copy": datetime("2011-09-16T22:24:17.000Z"), "friend-ids": {{ 19318451, 47946991, 1913830, 45324890, 47189256, 39211392, 6998884, 4344587, 24720830, 4355756, 19102058, 34241496, 39408673, 1360498, 7695088, 25754984, 21796436 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2007-09-19"), "end-date": date("2010-07-22") } ] }
+{ "id": 11918764, "id-copy": 11918764, "alias": "Jamison", "name": "JamisonKnight", "user-since": datetime("2012-02-28T12:46:09.000Z"), "user-since-copy": datetime("2012-02-28T12:46:09.000Z"), "friend-ids": {{ 5296309, 37783012, 18620712, 8255206, 10270999, 47361618, 39691488, 33528430, 22926601, 12751125, 34000354, 32638692, 19461108, 9760202, 30157968, 265361, 24683869, 19612648, 29021437, 40094162 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2003-04-16"), "end-date": date("2011-08-28") } ] }
+{ "id": 11920375, "id-copy": 11920375, "alias": "Terance", "name": "TeranceSaylor", "user-since": datetime("2005-02-09T10:33:47.000Z"), "user-since-copy": datetime("2005-02-09T10:33:47.000Z"), "friend-ids": {{ 17869677, 39051840, 6852335, 6153367, 1318628, 9983745, 5401091, 32798056, 42870494, 10337793, 43570623, 3233493, 38297525, 43712104, 15430099, 36703995, 25022620, 3681464, 21499719, 33737350, 6602331, 35391438, 47011233 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2005-11-05"), "end-date": date("2011-04-20") } ] }
+{ "id": 11988241, "id-copy": 11988241, "alias": "Cyrilla", "name": "CyrillaRohtin", "user-since": datetime("2005-02-10T08:24:14.000Z"), "user-since-copy": datetime("2005-02-10T08:24:14.000Z"), "friend-ids": {{ 32725541, 26677413, 29278988, 218049, 19833496, 20655804, 27991386, 5326490, 28583388, 41013948, 35541276, 41552165, 8526660 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2004-07-20"), "end-date": date("2004-08-19") } ] }
+{ "id": 11989645, "id-copy": 11989645, "alias": "Weston", "name": "WestonPershing", "user-since": datetime("2010-04-02T17:25:31.000Z"), "user-since-copy": datetime("2010-04-02T17:25:31.000Z"), "friend-ids": {{ 11689127 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2006-03-27"), "end-date": null } ] }
+{ "id": 11990740, "id-copy": 11990740, "alias": "Vernon", "name": "VernonBarnes", "user-since": datetime("2005-05-25T09:07:06.000Z"), "user-since-copy": datetime("2005-05-25T09:07:06.000Z"), "friend-ids": {{ 44677447, 20354746, 30157224, 29686873, 9413456, 11656099, 25404439, 24706566, 45005726, 22096097, 29868918, 12109246, 38948331, 2643312, 41565707, 17566751, 8045341, 25358960, 43614095, 28262168, 14405467, 22519550 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2012-01-23"), "end-date": null } ] }
+{ "id": 9025786, "id-copy": 9025786, "alias": "Terrance", "name": "TerranceFinlay", "user-since": datetime("2009-12-28T02:19:23.000Z"), "user-since-copy": datetime("2009-12-28T02:19:23.000Z"), "friend-ids": {{ 45324679, 13507068, 46678304, 37010727, 44866157, 12584675, 34305776, 14467180, 37751377, 2448873, 32584169, 14120838, 8902593, 31955437, 13436805 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2012-02-19"), "end-date": date("2012-07-25") } ] }
+{ "id": 9041578, "id-copy": 9041578, "alias": "Kristia", "name": "KristiaWillcox", "user-since": datetime("2012-01-09T10:29:02.000Z"), "user-since-copy": datetime("2012-01-09T10:29:02.000Z"), "friend-ids": {{ 29794000, 34515675, 3759231, 14418948, 35788028, 34225561, 20821065, 27582458, 4424723 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2005-06-04"), "end-date": date("2008-01-13") } ] }
+{ "id": 9041689, "id-copy": 9041689, "alias": "Freeman", "name": "FreemanDriggers", "user-since": datetime("2011-05-23T03:51:13.000Z"), "user-since-copy": datetime("2011-05-23T03:51:13.000Z"), "friend-ids": {{ 29448942, 29196543, 22725448, 15145190, 11938396, 44028947, 18379392, 21813464, 7448397, 43717728, 10728731, 24177517, 29069798, 37056934, 27601399, 26867839, 16593922, 22247111 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2007-01-14"), "end-date": null } ] }
+{ "id": 9042022, "id-copy": 9042022, "alias": "Fran", "name": "FranIronmonger", "user-since": datetime("2006-05-22T03:51:10.000Z"), "user-since-copy": datetime("2006-05-22T03:51:10.000Z"), "friend-ids": {{ 38546356, 31805246 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2002-06-06"), "end-date": null } ] }
+{ "id": 9056494, "id-copy": 9056494, "alias": "Alvena", "name": "AlvenaPearsall", "user-since": datetime("2005-08-09T08:50:25.000Z"), "user-since-copy": datetime("2005-08-09T08:50:25.000Z"), "friend-ids": {{ 26263956, 80589, 37669623, 32875186, 42026139, 22169384, 47224581, 25632957, 28392334, 42393204, 15028714, 554526 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2005-02-19"), "end-date": null } ] }
+{ "id": 9087292, "id-copy": 9087292, "alias": "Kiersten", "name": "KierstenRawls", "user-since": datetime("2005-03-21T08:42:24.000Z"), "user-since-copy": datetime("2005-03-21T08:42:24.000Z"), "friend-ids": {{ 5551555, 2958358, 17900476, 23956783, 31634897, 12573318, 32475113, 47343698, 40929064, 39881831, 38067700, 3519291, 19229024, 4383684, 13932328, 16414275, 8654888, 16145374, 26880764 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2004-03-10"), "end-date": null } ] }
+{ "id": 9125827, "id-copy": 9125827, "alias": "Kary", "name": "KaryHildyard", "user-since": datetime("2006-03-17T23:21:33.000Z"), "user-since-copy": datetime("2006-03-17T23:21:33.000Z"), "friend-ids": {{ 5570026 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2007-09-27"), "end-date": null } ] }
+{ "id": 9136882, "id-copy": 9136882, "alias": "Cassie", "name": "CassieGarratt", "user-since": datetime("2005-08-07T05:09:11.000Z"), "user-since-copy": datetime("2005-08-07T05:09:11.000Z"), "friend-ids": {{ 40916371, 42882703, 37748113, 45347468, 37653228, 15540626, 29276950, 31566687, 14600173, 12909057, 39561446, 41035377, 45987458, 43649639, 24488758, 25625568, 15566464, 584815, 35900688, 1079087, 46148561, 46404398 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2010-11-13"), "end-date": date("2010-09-04") } ] }
+{ "id": 9139966, "id-copy": 9139966, "alias": "Elwood", "name": "ElwoodDavis", "user-since": datetime("2009-04-25T20:38:07.000Z"), "user-since-copy": datetime("2009-04-25T20:38:07.000Z"), "friend-ids": {{ 28327906, 35534034, 3278109, 20721373, 40303614, 22594044, 3292862, 42117489, 18133788, 31771270, 43837818, 36567026 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2004-09-03"), "end-date": date("2011-07-03") } ] }
+{ "id": 9151357, "id-copy": 9151357, "alias": "Clover", "name": "CloverTedrow", "user-since": datetime("2012-04-04T22:46:03.000Z"), "user-since-copy": datetime("2012-04-04T22:46:03.000Z"), "friend-ids": {{ 47959325, 11808875, 46311157, 33138600, 15486362, 27921017, 32586367, 24379643, 14793815, 5841252, 22249573, 2147304, 47811082, 40329394, 4601822, 27977744, 45733056 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2002-09-07"), "end-date": date("2006-08-04") } ] }
+{ "id": 9174313, "id-copy": 9174313, "alias": "Hal", "name": "HalHasely", "user-since": datetime("2008-01-28T17:01:16.000Z"), "user-since-copy": datetime("2008-01-28T17:01:16.000Z"), "friend-ids": {{ 9058102, 40616538, 45706325, 991699, 37832260, 4793008, 36372035, 23272432, 36685642, 2621984, 9576806, 14325601, 41449409, 16499609, 20610820, 1564035, 20738111, 19735088, 31942764, 34813086 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2006-10-16"), "end-date": null } ] }
+{ "id": 9179122, "id-copy": 9179122, "alias": "Zach", "name": "ZachMilliron", "user-since": datetime("2011-07-28T01:09:04.000Z"), "user-since-copy": datetime("2011-07-28T01:09:04.000Z"), "friend-ids": {{ 40552138, 19204406, 46806031, 18794200, 45071131, 40119114, 23955279, 11126709, 37101358, 23332998, 1172034, 41496458, 32278235, 30949991, 148070, 6360227, 7378339, 33611217 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2000-06-26"), "end-date": null } ] }
+{ "id": 9190501, "id-copy": 9190501, "alias": "Leonardo", "name": "LeonardoBarr", "user-since": datetime("2008-02-23T14:20:45.000Z"), "user-since-copy": datetime("2008-02-23T14:20:45.000Z"), "friend-ids": {{ 24193096, 44367993, 10307197, 20420512, 36000544, 45069724, 42621729, 10863302, 21701700, 7110735, 6226449, 3269792, 12797617, 19460642, 7357145, 27051982, 31847212, 28691920, 382743, 11602175, 1787538, 42283089, 19610964 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2001-06-25"), "end-date": null } ] }
+{ "id": 9201610, "id-copy": 9201610, "alias": "Elaine", "name": "ElaineMcclymonds", "user-since": datetime("2008-04-13T17:06:35.000Z"), "user-since-copy": datetime("2008-04-13T17:06:35.000Z"), "friend-ids": {{ 18934024, 5114594, 25593808 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2006-08-28"), "end-date": null } ] }
+{ "id": 9216376, "id-copy": 9216376, "alias": "Stanford", "name": "StanfordBurney", "user-since": datetime("2010-04-24T23:03:06.000Z"), "user-since-copy": datetime("2010-04-24T23:03:06.000Z"), "friend-ids": {{ 15567770, 24839882, 163708, 45725879, 43621238, 27363995, 46782727, 21660511, 37585197, 17426559, 47247057, 41831246, 23944363, 1608826, 25831838, 41140458, 37108898, 19739056, 7475981, 17807472, 3126856, 40257768, 44873566 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2004-10-04"), "end-date": null } ] }
+{ "id": 9219955, "id-copy": 9219955, "alias": "Audrey", "name": "AudreyOmara", "user-since": datetime("2011-06-04T15:31:25.000Z"), "user-since-copy": datetime("2011-06-04T15:31:25.000Z"), "friend-ids": {{ 28209595, 29907721, 18295175, 18631813, 3380755, 20244076, 43026452, 42394327, 10914853, 27224999 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2003-03-24"), "end-date": null } ] }
+{ "id": 9233794, "id-copy": 9233794, "alias": "Jeffrey", "name": "JeffreyThrockmorton", "user-since": datetime("2005-04-23T04:24:31.000Z"), "user-since-copy": datetime("2005-04-23T04:24:31.000Z"), "friend-ids": {{ 29565308, 29107229, 35495609, 27358360, 24507795, 18583779, 16799427, 3571959, 6539875, 32120867, 17248402, 12227155, 37995559, 29425657, 20855502 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2000-04-22"), "end-date": date("2010-05-28") } ] }
+{ "id": 9243769, "id-copy": 9243769, "alias": "Florentino", "name": "FlorentinoRiggle", "user-since": datetime("2012-04-04T17:10:31.000Z"), "user-since-copy": datetime("2012-04-04T17:10:31.000Z"), "friend-ids": {{ 41929020, 22354873 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2007-07-23"), "end-date": null } ] }
+{ "id": 9267397, "id-copy": 9267397, "alias": "Corbin", "name": "CorbinWhite", "user-since": datetime("2006-01-07T07:43:27.000Z"), "user-since-copy": datetime("2006-01-07T07:43:27.000Z"), "friend-ids": {{ 11772390, 16826538, 16103166, 3256508, 40044263, 44187580, 29521314, 46200384, 40192445, 1239869, 14257012, 21632509, 6292478, 38738535, 18136574, 8369661, 45672754 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2000-09-16"), "end-date": date("2003-07-12") } ] }
+{ "id": 9269422, "id-copy": 9269422, "alias": "Roddy", "name": "RoddyFriedline", "user-since": datetime("2007-03-26T23:41:29.000Z"), "user-since-copy": datetime("2007-03-26T23:41:29.000Z"), "friend-ids": {{ 31923430, 19739952, 30983882, 10630795 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2001-06-03"), "end-date": null } ] }
+{ "id": 9271291, "id-copy": 9271291, "alias": "Kaitlynn", "name": "KaitlynnPycroft", "user-since": datetime("2010-10-09T11:30:12.000Z"), "user-since-copy": datetime("2010-10-09T11:30:12.000Z"), "friend-ids": {{ 38067939, 25732262, 17076819, 19477302, 29794559 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2006-09-04"), "end-date": null } ] }
+{ "id": 9274378, "id-copy": 9274378, "alias": "Callista", "name": "CallistaCatleay", "user-since": datetime("2012-01-11T05:02:51.000Z"), "user-since-copy": datetime("2012-01-11T05:02:51.000Z"), "friend-ids": {{ 35709258, 45469345, 7683235, 10959232, 44123341, 35853639, 11693773, 39944820, 47667622, 42781782, 4756825, 23566535 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2002-04-15"), "end-date": date("2003-04-03") } ] }
+{ "id": 9275620, "id-copy": 9275620, "alias": "Jackie", "name": "JackieRumbaugh", "user-since": datetime("2011-10-11T07:30:25.000Z"), "user-since-copy": datetime("2011-10-11T07:30:25.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2004-01-14"), "end-date": null } ] }
+{ "id": 9292738, "id-copy": 9292738, "alias": "Walter", "name": "WalterWain", "user-since": datetime("2012-05-03T10:41:22.000Z"), "user-since-copy": datetime("2012-05-03T10:41:22.000Z"), "friend-ids": {{ 1834068, 38777276, 43381631, 32380769, 23994313, 37459142, 21015234, 23788270, 33191448, 31111521, 21788604, 39349512, 20638072, 17300228, 4712935, 36205876, 27740958, 27236154 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2011-03-28"), "end-date": null } ] }
+{ "id": 9311659, "id-copy": 9311659, "alias": "Kate", "name": "KateBender", "user-since": datetime("2007-06-10T05:55:50.000Z"), "user-since-copy": datetime("2007-06-10T05:55:50.000Z"), "friend-ids": {{ 27875958, 10379355, 4286877, 26410945, 10609943, 15960135 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-01-04"), "end-date": null } ] }
+{ "id": 9317395, "id-copy": 9317395, "alias": "Timothy", "name": "TimothyMays", "user-since": datetime("2007-05-23T15:42:26.000Z"), "user-since-copy": datetime("2007-05-23T15:42:26.000Z"), "friend-ids": {{ 38066468, 16126194, 20685050, 8542551, 36810930, 36333903, 31522960, 44908120, 45171970, 9212095, 16986466, 41689196, 22300874, 45983009, 30918582, 5896299, 2682406, 6649020, 33199300, 14523848 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2006-04-16"), "end-date": date("2008-02-21") } ] }
+{ "id": 9320062, "id-copy": 9320062, "alias": "Samantha", "name": "SamanthaTanner", "user-since": datetime("2010-06-25T14:13:49.000Z"), "user-since-copy": datetime("2010-06-25T14:13:49.000Z"), "friend-ids": {{ 19538026 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2009-02-24"), "end-date": null } ] }
+{ "id": 9329746, "id-copy": 9329746, "alias": "Albert", "name": "AlbertZundel", "user-since": datetime("2005-11-01T23:41:02.000Z"), "user-since-copy": datetime("2005-11-01T23:41:02.000Z"), "friend-ids": {{ 44252308, 14483702, 27233282, 24263669, 35409140, 38591765, 42901786, 24502313, 6384822, 36359249, 36816246, 16578182, 530819, 29481837, 12698700, 6101521, 11990316, 35327955, 10435272 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2003-08-06"), "end-date": date("2010-09-22") } ] }
+{ "id": 9332161, "id-copy": 9332161, "alias": "Lavinia", "name": "LaviniaLineman", "user-since": datetime("2006-02-07T20:39:55.000Z"), "user-since-copy": datetime("2006-02-07T20:39:55.000Z"), "friend-ids": {{ 21419337, 31581364 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2012-07-05"), "end-date": null } ] }
+{ "id": 9341008, "id-copy": 9341008, "alias": "Gus", "name": "GusGearhart", "user-since": datetime("2012-05-23T13:19:57.000Z"), "user-since-copy": datetime("2012-05-23T13:19:57.000Z"), "friend-ids": {{ 20124243, 19722425, 20605718, 21833401, 18276801, 46184199, 40454562, 22828817, 44122338, 4485860, 34209581, 19783645, 44454238, 1353350, 37958534, 33547730, 2456119, 3023314, 44828467, 46655836, 33144170, 16864855, 41938662 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2002-07-07"), "end-date": null } ] }
+{ "id": 9343705, "id-copy": 9343705, "alias": "Ramsey", "name": "RamseyWarner", "user-since": datetime("2006-04-24T09:52:39.000Z"), "user-since-copy": datetime("2006-04-24T09:52:39.000Z"), "friend-ids": {{ 36909861, 36881715, 40993685, 18669519, 42428458, 2780280, 6070725, 10466662, 26215221, 16329040, 38464211, 14024902, 8083000, 27857433, 14282674, 1976238, 6345526, 35452338, 21503723, 34910137, 26860195, 426384, 27759959 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2003-10-28"), "end-date": null } ] }
+{ "id": 9345424, "id-copy": 9345424, "alias": "Jasmin", "name": "JasminGaskins", "user-since": datetime("2012-06-15T19:40:07.000Z"), "user-since-copy": datetime("2012-06-15T19:40:07.000Z"), "friend-ids": {{ 20837477, 42339634, 41136248, 24571549, 41060055, 18621328, 2057295, 41313707 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2012-05-27"), "end-date": date("2012-07-28") } ] }
+{ "id": 9356098, "id-copy": 9356098, "alias": "Juliana", "name": "JulianaAnderson", "user-since": datetime("2007-04-26T20:13:07.000Z"), "user-since-copy": datetime("2007-04-26T20:13:07.000Z"), "friend-ids": {{ 3850702, 46858392, 20177889, 34485932, 20958453, 26839176, 23562562, 47962945, 43961803, 19857248, 29816714, 14695228, 35327929, 16196977, 11908428, 30035277, 23919929 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2009-06-04"), "end-date": date("2009-05-05") } ] }
+{ "id": 9367306, "id-copy": 9367306, "alias": "Jacinth", "name": "JacinthBynum", "user-since": datetime("2012-03-08T11:26:04.000Z"), "user-since-copy": datetime("2012-03-08T11:26:04.000Z"), "friend-ids": {{ 35048012, 42620612, 39526901, 12673410, 16363143, 45509270, 47714729, 47902094, 12551745, 45510597, 31513255, 2848992, 16088751, 1953590, 32956014, 38607548, 15982103, 31161780, 7331812, 44977526, 15022020, 19905573 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2003-03-24"), "end-date": null } ] }
+{ "id": 9369847, "id-copy": 9369847, "alias": "Jeffrey", "name": "JeffreyArchibald", "user-since": datetime("2011-07-11T23:43:52.000Z"), "user-since-copy": datetime("2011-07-11T23:43:52.000Z"), "friend-ids": {{ 44928062, 45653705 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2010-03-25"), "end-date": null } ] }
+{ "id": 9386794, "id-copy": 9386794, "alias": "Issac", "name": "IssacNickolson", "user-since": datetime("2009-12-11T08:40:10.000Z"), "user-since-copy": datetime("2009-12-11T08:40:10.000Z"), "friend-ids": {{ 4077760, 26197904, 22088648 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2002-05-17"), "end-date": null } ] }
+{ "id": 9389254, "id-copy": 9389254, "alias": "Jon", "name": "JonShaw", "user-since": datetime("2006-12-10T11:28:23.000Z"), "user-since-copy": datetime("2006-12-10T11:28:23.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2010-07-24"), "end-date": null } ] }
+{ "id": 9396193, "id-copy": 9396193, "alias": "Franklyn", "name": "FranklynVorrasi", "user-since": datetime("2007-06-27T09:38:03.000Z"), "user-since-copy": datetime("2007-06-27T09:38:03.000Z"), "friend-ids": {{ 12870114, 28811462, 19219273, 38745339, 22310708, 11419733, 21583164, 42276545, 1177024, 43617748, 11702666, 19332437, 1523883, 40265275, 41227772 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2001-03-13"), "end-date": date("2009-02-07") } ] }
+{ "id": 9415921, "id-copy": 9415921, "alias": "Shad", "name": "ShadHaynes", "user-since": datetime("2010-01-19T22:19:28.000Z"), "user-since-copy": datetime("2010-01-19T22:19:28.000Z"), "friend-ids": {{ 4608515, 39839555, 31370710, 43278478, 731705, 26523982, 15560444, 10605444, 20229128, 41477079, 47960417, 1744587, 35477897, 10362849, 38394199, 24090076, 14390416 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2010-06-23"), "end-date": null } ] }
+{ "id": 9430849, "id-copy": 9430849, "alias": "Emil", "name": "EmilGarland", "user-since": datetime("2008-07-03T15:56:07.000Z"), "user-since-copy": datetime("2008-07-03T15:56:07.000Z"), "friend-ids": {{ 40429008, 45432330, 22293451, 2129366, 19514477, 20108162, 28656704, 35403173, 33855801, 14660181 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2010-02-10"), "end-date": null } ] }
+{ "id": 9434542, "id-copy": 9434542, "alias": "Alice", "name": "AliceRopes", "user-since": datetime("2011-09-10T10:32:17.000Z"), "user-since-copy": datetime("2011-09-10T10:32:17.000Z"), "friend-ids": {{ 30233815, 23593045, 243865, 46494768, 15852416, 2627657, 12253908, 11415849, 36381160, 25773586, 9952015, 20363967, 45499740, 15573031, 2939342, 24137982, 34026341, 34111551, 30963526, 7116453 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2008-07-09"), "end-date": null } ] }
+{ "id": 9453925, "id-copy": 9453925, "alias": "Ritchie", "name": "RitchieJube", "user-since": datetime("2008-04-28T12:33:34.000Z"), "user-since-copy": datetime("2008-04-28T12:33:34.000Z"), "friend-ids": {{ 44327769, 45189889, 11098478, 41612069, 40647950, 638474, 21614810, 22273745, 6230791, 15120137, 18477729, 16895919, 5907839, 43993812, 31639138, 7966991, 11024409 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2012-07-22"), "end-date": null } ] }
+{ "id": 9461770, "id-copy": 9461770, "alias": "Georgina", "name": "GeorginaPearson", "user-since": datetime("2005-02-04T09:47:21.000Z"), "user-since-copy": datetime("2005-02-04T09:47:21.000Z"), "friend-ids": {{ 26615251, 5874803, 5189465, 29564778, 1778424, 38706542, 38915757, 16819394, 3318129, 2166806, 30570432, 15192853, 4857015, 41673300, 23510020 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2003-06-06"), "end-date": null } ] }
+{ "id": 9477919, "id-copy": 9477919, "alias": "Lilly", "name": "LillyLinton", "user-since": datetime("2005-01-09T12:24:01.000Z"), "user-since-copy": datetime("2005-01-09T12:24:01.000Z"), "friend-ids": {{ 19117935, 45208482, 36019625, 39146688, 15911832 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2011-03-03"), "end-date": date("2011-10-03") } ] }
+{ "id": 9497698, "id-copy": 9497698, "alias": "Jenny", "name": "JennyBiery", "user-since": datetime("2007-07-24T17:20:06.000Z"), "user-since-copy": datetime("2007-07-24T17:20:06.000Z"), "friend-ids": {{ 37832227, 17148339, 38184683, 45775690, 17511050, 1866913, 30631091, 5996302, 3796747, 33135567, 5930972, 9509054, 44003369, 34299276, 16135297, 15435466, 42464299, 34961792, 47264306, 30734198, 26192613 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2007-10-02"), "end-date": date("2011-09-20") } ] }
+{ "id": 9503761, "id-copy": 9503761, "alias": "Demelza", "name": "DemelzaLaw", "user-since": datetime("2010-12-17T06:40:19.000Z"), "user-since-copy": datetime("2010-12-17T06:40:19.000Z"), "friend-ids": {{ 34126746, 5537488, 609154, 35877951, 36237612 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2005-10-22"), "end-date": null } ] }
+{ "id": 9512971, "id-copy": 9512971, "alias": "Algar", "name": "AlgarKepplinger", "user-since": datetime("2011-10-11T02:54:01.000Z"), "user-since-copy": datetime("2011-10-11T02:54:01.000Z"), "friend-ids": {{ 1076656, 1837449, 43428033, 21710004, 41167492, 17526252 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2003-04-26"), "end-date": date("2006-02-24") } ] }
+{ "id": 9512989, "id-copy": 9512989, "alias": "Lilliana", "name": "LillianaAdams", "user-since": datetime("2007-06-01T16:54:29.000Z"), "user-since-copy": datetime("2007-06-01T16:54:29.000Z"), "friend-ids": {{ 14085316, 47471900, 24950195, 44416851, 6677091, 34188319, 1783776, 35860593, 29193624, 11999697, 13365419, 39452732, 14401842, 9087264, 15679216, 39424118, 45063958, 11967959, 29634503, 15763396 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2001-02-08"), "end-date": date("2008-03-23") } ] }
+{ "id": 9516883, "id-copy": 9516883, "alias": "Delsie", "name": "DelsieKuster", "user-since": datetime("2005-11-20T06:18:01.000Z"), "user-since-copy": datetime("2005-11-20T06:18:01.000Z"), "friend-ids": {{ 7211399, 31355269, 10052966, 11255272, 11976144, 13036749, 28228775, 3501290, 35668522, 21064471, 47089958, 20725508, 16768149, 39282691, 44096922, 12469594, 8258231, 36373387, 14994345, 32022989, 28975684, 29840860 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2008-11-11"), "end-date": date("2008-03-06") } ] }
+{ "id": 9518128, "id-copy": 9518128, "alias": "Jerrie", "name": "JerrieFonblanque", "user-since": datetime("2008-06-08T02:51:53.000Z"), "user-since-copy": datetime("2008-06-08T02:51:53.000Z"), "friend-ids": {{ 41051692, 21547608, 41749297, 21528434, 28012731, 43579854, 9172140, 17908381, 10276804, 12277383, 38454166, 6950146, 11878198, 24415804, 46218827, 33013212, 44735001, 36395459, 38515534, 16015324, 21085620, 20338207 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2001-01-14"), "end-date": null } ] }
+{ "id": 9521683, "id-copy": 9521683, "alias": "Tennille", "name": "TennilleHamilton", "user-since": datetime("2009-04-21T20:56:25.000Z"), "user-since-copy": datetime("2009-04-21T20:56:25.000Z"), "friend-ids": {{ 32048407, 3619952, 41652292, 45570368, 31678290, 11241324 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2010-10-15"), "end-date": null } ] }
+{ "id": 9546133, "id-copy": 9546133, "alias": "Renae", "name": "RenaeWhitehead", "user-since": datetime("2012-04-21T14:38:30.000Z"), "user-since-copy": datetime("2012-04-21T14:38:30.000Z"), "friend-ids": {{ 31261211, 19892104, 35568606, 12050300, 42512152, 37032282, 27185051 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2012-02-20"), "end-date": date("2012-07-04") } ] }
+{ "id": 9563056, "id-copy": 9563056, "alias": "Iantha", "name": "IanthaHoward", "user-since": datetime("2009-03-09T10:16:12.000Z"), "user-since-copy": datetime("2009-03-09T10:16:12.000Z"), "friend-ids": {{ 31445918, 39207727, 45365035, 7861010, 28533268, 29009652, 40156013, 40416479, 42741676, 30221879, 30189614, 46450645, 30914117, 33681301, 19457868, 23309378, 15126664, 32913981, 5396205 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2000-03-18"), "end-date": date("2009-01-05") } ] }
+{ "id": 9602242, "id-copy": 9602242, "alias": "Marc", "name": "MarcDimsdale", "user-since": datetime("2005-10-03T23:32:18.000Z"), "user-since-copy": datetime("2005-10-03T23:32:18.000Z"), "friend-ids": {{ 34004502, 24469994, 2140538, 1486939, 6895407, 13348535, 22384921, 11662871, 21398307, 33070732, 45602509, 26146770, 24148813, 45988030, 22184030, 855669, 36390708, 30883354, 26360628, 29836897, 28696575 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2004-05-15"), "end-date": date("2008-01-19") } ] }
+{ "id": 9634393, "id-copy": 9634393, "alias": "Burt", "name": "BurtPearson", "user-since": datetime("2007-11-01T14:25:29.000Z"), "user-since-copy": datetime("2007-11-01T14:25:29.000Z"), "friend-ids": {{ 26065414, 8710639, 22639162, 23787625, 24443211, 42598742, 45171006, 38246985, 25125478, 23071168, 22455706, 24720860, 34917747, 24262081, 2259812, 14262605, 37533604 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-05-07"), "end-date": null } ] }
+{ "id": 9636802, "id-copy": 9636802, "alias": "Gage", "name": "GageHair", "user-since": datetime("2011-01-23T22:31:49.000Z"), "user-since-copy": datetime("2011-01-23T22:31:49.000Z"), "friend-ids": {{ 46795684, 38195763, 25882078, 28871879, 5178144, 17683475, 43441471, 5427133, 13936915, 2608474, 9513798, 31041524, 557454, 22452168, 12948004, 16835098, 1151241, 37188687 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2000-05-02"), "end-date": date("2010-02-13") } ] }
+{ "id": 9640915, "id-copy": 9640915, "alias": "Harrison", "name": "HarrisonHildyard", "user-since": datetime("2009-05-25T11:56:05.000Z"), "user-since-copy": datetime("2009-05-25T11:56:05.000Z"), "friend-ids": {{ 41488832, 16139664, 18327029, 38811764, 38271538, 13106137, 26450611, 11574808, 33108523, 31639017, 9208159, 18456510, 47955463, 2606160, 29293146, 13981743, 39967993, 23629640, 32666499, 35046044, 2402842, 1117025, 17741007, 14997808 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2010-03-06"), "end-date": null } ] }
+{ "id": 9643768, "id-copy": 9643768, "alias": "Gil", "name": "GilVeith", "user-since": datetime("2006-04-26T11:42:30.000Z"), "user-since-copy": datetime("2006-04-26T11:42:30.000Z"), "friend-ids": {{ 22270431, 9614818, 9080111, 6500797, 37876717, 28122656, 13971193, 20936637, 19883735, 37455193, 32129291, 40710966, 17779823, 41523128, 41276564, 34424817, 19326867, 26058281 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2002-02-04"), "end-date": null } ] }
+{ "id": 9665848, "id-copy": 9665848, "alias": "Shannah", "name": "ShannahDale", "user-since": datetime("2006-08-09T02:09:51.000Z"), "user-since-copy": datetime("2006-08-09T02:09:51.000Z"), "friend-ids": {{ 19512022, 25217933, 21742776, 35558948, 5893317, 2441637, 6907563, 36626257, 3366834, 25069218, 5753530, 45604388, 33908296, 1048890, 5720452, 7923351, 43424884, 43184720, 29744229, 10349400, 15273614, 15283237, 41997307 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2010-12-28"), "end-date": date("2010-09-17") } ] }
+{ "id": 9676201, "id-copy": 9676201, "alias": "Jessica", "name": "JessicaBeals", "user-since": datetime("2006-12-02T17:13:07.000Z"), "user-since-copy": datetime("2006-12-02T17:13:07.000Z"), "friend-ids": {{ 40180348, 5499689, 43937013, 12294744, 47607871, 15173594, 19403387, 30591667, 1488569, 11862843, 26230465, 15334606, 4397778, 8140277, 39859715, 25854759, 7216524, 41695061, 43036500, 15618315, 4503056, 23790965, 14510949, 34347866 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2011-05-15"), "end-date": date("2011-10-27") } ] }
+{ "id": 9677293, "id-copy": 9677293, "alias": "Owen", "name": "OwenHoenshell", "user-since": datetime("2005-06-28T02:54:49.000Z"), "user-since-copy": datetime("2005-06-28T02:54:49.000Z"), "friend-ids": {{ 1016713, 4999321, 27107303, 15587298 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2010-01-11"), "end-date": null } ] }
+{ "id": 9683656, "id-copy": 9683656, "alias": "Antone", "name": "AntoneMays", "user-since": datetime("2006-07-24T22:48:29.000Z"), "user-since-copy": datetime("2006-07-24T22:48:29.000Z"), "friend-ids": {{ 11275116, 40325672, 41154035, 8987353, 31187312, 11505721, 11584703, 42743337, 23225356, 8653923 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2011-06-12"), "end-date": null } ] }
+{ "id": 9695773, "id-copy": 9695773, "alias": "Daron", "name": "DaronFiddler", "user-since": datetime("2006-12-25T17:08:50.000Z"), "user-since-copy": datetime("2006-12-25T17:08:50.000Z"), "friend-ids": {{ 14397778, 33469556, 41690231, 7827360, 42196316 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2007-12-24"), "end-date": null } ] }
+{ "id": 9698980, "id-copy": 9698980, "alias": "Leland", "name": "LelandReiss", "user-since": datetime("2012-05-23T04:40:29.000Z"), "user-since-copy": datetime("2012-05-23T04:40:29.000Z"), "friend-ids": {{ 7623016, 12672253, 42612513, 44457047, 46981337, 1098470, 23122899, 15019916, 45345438, 30272843, 43546610 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2001-11-27"), "end-date": null } ] }
+{ "id": 9699673, "id-copy": 9699673, "alias": "Jim", "name": "JimPycroft", "user-since": datetime("2012-07-25T20:20:38.000Z"), "user-since-copy": datetime("2012-07-25T20:20:38.000Z"), "friend-ids": {{ 14858146, 47543880, 3186927, 38198580, 2365336, 5255886, 11178580, 41188272, 17623582, 6422949, 4405751, 12128017, 32409443, 38861849, 16511892, 24515731, 46665640, 40644816, 19341995, 44288533, 26148671 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2007-01-24"), "end-date": date("2009-12-16") } ] }
+{ "id": 9707074, "id-copy": 9707074, "alias": "Melvyn", "name": "MelvynSybilla", "user-since": datetime("2012-06-07T16:06:49.000Z"), "user-since-copy": datetime("2012-06-07T16:06:49.000Z"), "friend-ids": {{ 4487400, 488933, 15650706, 44692005, 25068052, 16975927 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2010-12-13"), "end-date": null } ] }
+{ "id": 9740008, "id-copy": 9740008, "alias": "Woodrow", "name": "WoodrowBlois", "user-since": datetime("2011-12-18T11:34:56.000Z"), "user-since-copy": datetime("2011-12-18T11:34:56.000Z"), "friend-ids": {{ 1753941, 17603348, 44569557, 6816408, 17403631, 29707555, 21215516, 9837919, 35887854, 35236051, 7897485, 9880491, 16145458, 33128036, 41471362, 44171952, 23542112, 36155237, 2596261, 36702766 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2007-08-02"), "end-date": null } ] }
+{ "id": 9752227, "id-copy": 9752227, "alias": "Audley", "name": "AudleyPeters", "user-since": datetime("2006-07-27T01:15:35.000Z"), "user-since-copy": datetime("2006-07-27T01:15:35.000Z"), "friend-ids": {{ 877448, 29611844, 2844046, 42493473, 28216181, 353847, 44172105, 36184409, 44010617 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2002-12-17"), "end-date": null } ] }
+{ "id": 9765517, "id-copy": 9765517, "alias": "Alexia", "name": "AlexiaTownsend", "user-since": datetime("2006-02-23T13:26:33.000Z"), "user-since-copy": datetime("2006-02-23T13:26:33.000Z"), "friend-ids": {{ 39892441, 43413199, 45070224, 46877180, 24247279, 26450737, 29111107, 46768934, 11833332, 25913646, 43063781 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2003-02-08"), "end-date": null } ] }
+{ "id": 9769501, "id-copy": 9769501, "alias": "Geffrey", "name": "GeffreyBurch", "user-since": datetime("2005-08-28T03:10:56.000Z"), "user-since-copy": datetime("2005-08-28T03:10:56.000Z"), "friend-ids": {{ 21060169, 45384418, 20564855, 24708101, 30231, 29383832, 9200835, 822161, 29674263, 619991, 38797966, 14299510, 13545166, 33027152 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2004-01-03"), "end-date": date("2006-04-13") } ] }
+{ "id": 9773836, "id-copy": 9773836, "alias": "Harris", "name": "HarrisAshmore", "user-since": datetime("2005-11-09T08:38:57.000Z"), "user-since-copy": datetime("2005-11-09T08:38:57.000Z"), "friend-ids": {{ 8138978, 18579002, 42663609, 12096643, 38992166, 36937135, 19634600, 2103929, 37072923, 25031081, 13379299, 11238246, 23166598, 19181943, 45382447, 8237252, 30986231, 29591835 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2000-02-12"), "end-date": null } ] }
+{ "id": 9804196, "id-copy": 9804196, "alias": "Micheal", "name": "MichealEiford", "user-since": datetime("2009-05-21T02:55:17.000Z"), "user-since-copy": datetime("2009-05-21T02:55:17.000Z"), "friend-ids": {{ 31376257, 19749408, 5790154, 17891222, 15712036, 40911870, 40765983, 38804584, 24619388, 10957577, 35370581, 39352927, 6063001, 23702369, 14716580, 46589395, 35232946 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2003-07-27"), "end-date": null } ] }
+{ "id": 9811513, "id-copy": 9811513, "alias": "Casie", "name": "CasieRose", "user-since": datetime("2011-11-25T11:32:36.000Z"), "user-since-copy": datetime("2011-11-25T11:32:36.000Z"), "friend-ids": {{ 8913855, 26924028, 19426899, 38037518, 39689117, 32691982, 6561788, 36463261, 31724455, 18356325, 23130893, 35227626, 13738524, 4700460, 6963740, 13255939, 12215189, 33593825, 34229322 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2003-11-22"), "end-date": null } ] }
+{ "id": 9820681, "id-copy": 9820681, "alias": "Caitlin", "name": "CaitlinWolfe", "user-since": datetime("2012-05-23T07:59:39.000Z"), "user-since-copy": datetime("2012-05-23T07:59:39.000Z"), "friend-ids": {{ 22005473, 7664709, 22913945, 16078115, 11724028, 45958589, 33357270, 6935384, 2696233, 28938665, 37992833, 11929142, 16203505, 20365802 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2010-06-18"), "end-date": null } ] }
+{ "id": 9826402, "id-copy": 9826402, "alias": "Rachyl", "name": "RachylRumbaugh", "user-since": datetime("2006-01-05T03:38:59.000Z"), "user-since-copy": datetime("2006-01-05T03:38:59.000Z"), "friend-ids": {{ 11891915, 15900581, 38420311, 21084667, 24569500, 9181299, 32167823, 9967774, 18138704, 10742133, 29173609, 1113683, 21048344, 33794587, 42308958, 9303744 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2005-04-24"), "end-date": date("2008-08-17") } ] }
+{ "id": 9845113, "id-copy": 9845113, "alias": "Chia", "name": "ChiaGeddinge", "user-since": datetime("2008-12-12T16:50:57.000Z"), "user-since-copy": datetime("2008-12-12T16:50:57.000Z"), "friend-ids": {{ 16725476, 120161, 762756, 40795640, 34195102, 27938737 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2001-01-03"), "end-date": date("2001-11-03") } ] }
+{ "id": 9854788, "id-copy": 9854788, "alias": "Mathilda", "name": "MathildaVanleer", "user-since": datetime("2007-01-05T08:45:07.000Z"), "user-since-copy": datetime("2007-01-05T08:45:07.000Z"), "friend-ids": {{ 20510022, 1353061, 24801201, 11438611, 30281530, 15596343, 29404248, 2024925, 3425369, 18530400 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2002-07-22"), "end-date": date("2011-02-24") } ] }
+{ "id": 9859726, "id-copy": 9859726, "alias": "Taryn", "name": "TarynGisiko", "user-since": datetime("2010-12-28T21:42:56.000Z"), "user-since-copy": datetime("2010-12-28T21:42:56.000Z"), "friend-ids": {{ 45036313, 47860435, 40658528, 4106429, 25411752, 7216290, 20549107, 28317961, 43600081, 6359672, 36131464, 19078372, 4379305, 884797, 11605059, 6467240, 23316141 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2007-06-26"), "end-date": date("2010-08-04") } ] }
+{ "id": 9866572, "id-copy": 9866572, "alias": "Evelina", "name": "EvelinaBerry", "user-since": datetime("2006-12-16T03:56:00.000Z"), "user-since-copy": datetime("2006-12-16T03:56:00.000Z"), "friend-ids": {{ 13883615, 43198063, 30615747, 3228427, 23840450, 43443245, 17107485, 34691909, 44890462, 47992198, 46475465, 28790498, 7693182, 41338502, 6694688, 17592193, 9966336, 40899188, 16363000, 43996364 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2004-03-01"), "end-date": date("2008-08-21") } ] }
+{ "id": 9867190, "id-copy": 9867190, "alias": "Elvis", "name": "ElvisBasinger", "user-since": datetime("2009-01-16T11:48:43.000Z"), "user-since-copy": datetime("2009-01-16T11:48:43.000Z"), "friend-ids": {{ 31562017, 45465097, 29858836, 21720764, 37465930, 20639296, 7168709 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2011-01-11"), "end-date": date("2011-01-26") } ] }
+{ "id": 9880696, "id-copy": 9880696, "alias": "Cynthia", "name": "CynthiaSeidner", "user-since": datetime("2006-03-17T01:36:33.000Z"), "user-since-copy": datetime("2006-03-17T01:36:33.000Z"), "friend-ids": {{ 47318799, 28282167 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2008-07-02"), "end-date": date("2010-11-25") } ] }
+{ "id": 9882241, "id-copy": 9882241, "alias": "Dillon", "name": "DillonSimpson", "user-since": datetime("2006-03-20T13:21:16.000Z"), "user-since-copy": datetime("2006-03-20T13:21:16.000Z"), "friend-ids": {{ 22747996, 6266176, 22832223, 30880579, 35481343, 48005259, 381757, 27560756, 6053858, 42532723, 33355330, 40374460, 39019469, 35869327 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2007-06-13"), "end-date": date("2011-08-15") } ] }
+{ "id": 9883165, "id-copy": 9883165, "alias": "Dean", "name": "DeanKern", "user-since": datetime("2005-11-02T13:10:37.000Z"), "user-since-copy": datetime("2005-11-02T13:10:37.000Z"), "friend-ids": {{ 33343261, 27280204, 31345192, 723310, 11949431, 4787422, 28427922, 11974873, 24553234, 19067609, 12178905, 38171944, 26832701, 47422914, 47782561, 26391811, 28206950, 17135029, 37069726, 40613638, 11509775 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2003-02-16"), "end-date": date("2009-12-16") } ] }
+{ "id": 9885289, "id-copy": 9885289, "alias": "Kayla", "name": "KaylaDugger", "user-since": datetime("2007-10-20T12:55:38.000Z"), "user-since-copy": datetime("2007-10-20T12:55:38.000Z"), "friend-ids": {{ 1821427, 46609485, 4532131 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2009-02-15"), "end-date": date("2009-11-17") } ] }
+{ "id": 9890854, "id-copy": 9890854, "alias": "Linwood", "name": "LinwoodBrown", "user-since": datetime("2005-09-09T12:38:00.000Z"), "user-since-copy": datetime("2005-09-09T12:38:00.000Z"), "friend-ids": {{ 13728190, 31562633, 3437344, 13841675, 38528685 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2006-05-08"), "end-date": date("2009-08-26") } ] }
+{ "id": 9896473, "id-copy": 9896473, "alias": "Harlan", "name": "HarlanAnderson", "user-since": datetime("2012-06-03T22:40:33.000Z"), "user-since-copy": datetime("2012-06-03T22:40:33.000Z"), "friend-ids": {{ 28073049, 32365932, 23795268, 7563960, 47274822, 4907078, 8659018, 33480175, 3984139, 20631025, 26879093, 27168884, 20063035, 22192716, 18259756, 28904415, 28492528, 4140983, 12014021, 10959797, 38881978, 45835171, 6556552, 26372018 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2000-08-18"), "end-date": null } ] }
+{ "id": 9897094, "id-copy": 9897094, "alias": "Raynard", "name": "RaynardWade", "user-since": datetime("2010-05-12T19:44:55.000Z"), "user-since-copy": datetime("2010-05-12T19:44:55.000Z"), "friend-ids": {{ 21246472, 34504200, 43744110, 30518742, 1016046, 17644601, 47173648, 11643135, 22382871, 38535297, 17156487, 30328939, 14770807, 9365820, 36893585, 30122942, 37610936, 44304872 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2004-02-18"), "end-date": null } ] }
+{ "id": 9910003, "id-copy": 9910003, "alias": "Arline", "name": "ArlineElinor", "user-since": datetime("2012-07-20T16:57:36.000Z"), "user-since-copy": datetime("2012-07-20T16:57:36.000Z"), "friend-ids": {{ 34121202, 19342891, 45323168, 17272278, 6471047, 3726738, 48003127, 32423724, 38588754, 44816854, 13688032, 12876442 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2009-07-19"), "end-date": date("2009-04-17") } ] }
+{ "id": 9922381, "id-copy": 9922381, "alias": "Cecilia", "name": "CeciliaOsteen", "user-since": datetime("2009-06-03T03:58:36.000Z"), "user-since-copy": datetime("2009-06-03T03:58:36.000Z"), "friend-ids": {{ 22246989, 9095240, 8953245, 16326669, 38845534, 13608449, 35076758, 42004583 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2010-08-02"), "end-date": null } ] }
+{ "id": 9934939, "id-copy": 9934939, "alias": "Camilla", "name": "CamillaRhinehart", "user-since": datetime("2008-12-06T10:44:45.000Z"), "user-since-copy": datetime("2008-12-06T10:44:45.000Z"), "friend-ids": {{ 17020237, 36188716, 32765819, 20068359, 23060675, 16692600 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2012-04-05"), "end-date": null } ] }
+{ "id": 9945208, "id-copy": 9945208, "alias": "Thelma", "name": "ThelmaGettemy", "user-since": datetime("2006-12-21T11:17:06.000Z"), "user-since-copy": datetime("2006-12-21T11:17:06.000Z"), "friend-ids": {{ 26578648, 43730418, 18099472, 11787057, 41534206, 16778979, 41142786, 25761045, 18556835, 25378849, 38984390, 37528215, 2531696 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2008-03-25"), "end-date": null } ] }
+{ "id": 9958378, "id-copy": 9958378, "alias": "Floyd", "name": "FloydErrett", "user-since": datetime("2006-07-06T02:51:46.000Z"), "user-since-copy": datetime("2006-07-06T02:51:46.000Z"), "friend-ids": {{ 38108839, 44502073, 19244279, 45055684, 32489890, 25184431, 34275591, 47288414, 46973922, 28264345, 10024409, 4791958, 40576138, 33446414, 359486, 25595793, 25140170, 23149057, 47032976, 4283407 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2001-08-08"), "end-date": null } ] }
+{ "id": 9959077, "id-copy": 9959077, "alias": "Josephine", "name": "JosephineLauffer", "user-since": datetime("2006-12-27T17:33:39.000Z"), "user-since-copy": datetime("2006-12-27T17:33:39.000Z"), "friend-ids": {{ 41423014, 33024139, 26147665, 14776436, 4726952, 12688804 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2001-03-26"), "end-date": null } ] }
+{ "id": 9975778, "id-copy": 9975778, "alias": "Marmaduke", "name": "MarmadukeElizabeth", "user-since": datetime("2012-07-18T02:21:55.000Z"), "user-since-copy": datetime("2012-07-18T02:21:55.000Z"), "friend-ids": {{ 17424696, 34807936, 8912699, 40534595, 36049658, 31706902, 7626256, 16178188, 36944385, 47878361, 8190132, 34365280, 13576207, 42728095 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2006-08-19"), "end-date": null } ] }
+{ "id": 9985393, "id-copy": 9985393, "alias": "Whitaker", "name": "WhitakerMang", "user-since": datetime("2007-11-28T09:34:34.000Z"), "user-since-copy": datetime("2007-11-28T09:34:34.000Z"), "friend-ids": {{ 24107735, 37165967, 31305236, 35313360, 9261860, 32724193, 34416346, 8143882, 9029425, 26723829, 4545824 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2000-08-23"), "end-date": date("2008-08-06") } ] }
+{ "id": 10001047, "id-copy": 10001047, "alias": "Rodger", "name": "RodgerRifler", "user-since": datetime("2009-12-08T18:34:21.000Z"), "user-since-copy": datetime("2009-12-08T18:34:21.000Z"), "friend-ids": {{ 41832587, 41015556, 17486735, 38428485, 29774516, 38574837, 2061546, 46972940, 25654449, 776023, 1164809, 34242171, 9752352, 1088591, 26406961, 7270316, 36371574, 24413303, 36287374, 43343719, 6830709, 2919772, 41313339 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2003-02-08"), "end-date": null } ] }
+{ "id": 10001080, "id-copy": 10001080, "alias": "Garrett", "name": "GarrettBode", "user-since": datetime("2005-10-25T18:07:35.000Z"), "user-since-copy": datetime("2005-10-25T18:07:35.000Z"), "friend-ids": {{ 35858744, 16426061, 11473961, 4769664, 29038930, 33070686, 46271872, 42593454, 36202882, 46642640, 22243678, 20222041, 29014540, 7389258, 7172909, 12787979, 146736, 21081030, 21615179, 2936936, 44934891 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2007-06-24"), "end-date": null } ] }
+{ "id": 10025086, "id-copy": 10025086, "alias": "Peggy", "name": "PeggyOlphert", "user-since": datetime("2009-06-24T16:14:48.000Z"), "user-since-copy": datetime("2009-06-24T16:14:48.000Z"), "friend-ids": {{ 13659719, 46045788, 35841713, 32392118, 24785179, 45483286, 47287227, 42691471, 7471992, 47671331, 25747076, 2368606, 34452743, 14570607, 31436760, 36423303, 31381129, 29414651, 10005587, 14082638, 13311890, 11592210, 1585557 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2008-07-20"), "end-date": null } ] }
+{ "id": 10047001, "id-copy": 10047001, "alias": "Darcy", "name": "DarcyKava", "user-since": datetime("2012-02-25T17:16:18.000Z"), "user-since-copy": datetime("2012-02-25T17:16:18.000Z"), "friend-ids": {{ 15613341, 46557569, 20439965, 22442508, 32423739, 40757483, 36365324, 40706148, 12537361, 47741886, 24508947, 34168899, 10674474, 34285157, 28222068, 11113263 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2008-01-27"), "end-date": null } ] }
+{ "id": 10047373, "id-copy": 10047373, "alias": "Rexana", "name": "RexanaDennis", "user-since": datetime("2010-01-05T15:43:34.000Z"), "user-since-copy": datetime("2010-01-05T15:43:34.000Z"), "friend-ids": {{ 1594, 40130182 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2004-07-04"), "end-date": date("2007-12-28") } ] }
+{ "id": 10059343, "id-copy": 10059343, "alias": "Randy", "name": "RandyQueer", "user-since": datetime("2005-06-01T02:30:35.000Z"), "user-since-copy": datetime("2005-06-01T02:30:35.000Z"), "friend-ids": {{ 8688755, 7077909, 41009273, 26932559, 29488059, 6408736, 6374592, 5042147, 21880854, 12704496, 28046022, 2384964, 20867794, 3990470, 7132171 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2006-07-07"), "end-date": date("2007-04-08") } ] }
+{ "id": 10065595, "id-copy": 10065595, "alias": "Zenobia", "name": "ZenobiaHiggens", "user-since": datetime("2009-11-06T11:19:47.000Z"), "user-since-copy": datetime("2009-11-06T11:19:47.000Z"), "friend-ids": {{ 19623415, 12770212, 30381171, 20436392, 33497094, 39556081, 22592010, 44832685, 35801007, 39682093, 26870566, 8667589, 43790411, 24760722, 8286108, 20709133 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2001-07-28"), "end-date": date("2004-12-26") } ] }
+{ "id": 10071475, "id-copy": 10071475, "alias": "Kyra", "name": "KyraWile", "user-since": datetime("2010-08-21T20:27:23.000Z"), "user-since-copy": datetime("2010-08-21T20:27:23.000Z"), "friend-ids": {{ 24326501, 3159228, 33973593, 47221189, 17474184, 17812891 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2010-04-14"), "end-date": null } ] }
+{ "id": 10114891, "id-copy": 10114891, "alias": "Destinee", "name": "DestineeLeech", "user-since": datetime("2006-06-05T09:32:17.000Z"), "user-since-copy": datetime("2006-06-05T09:32:17.000Z"), "friend-ids": {{ 9925448, 28685906, 3305693, 11131758, 10477741, 19058196, 25921997, 38543939, 20851041 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2001-09-24"), "end-date": null } ] }
+{ "id": 10128076, "id-copy": 10128076, "alias": "Parker", "name": "ParkerHutton", "user-since": datetime("2011-06-05T03:46:01.000Z"), "user-since-copy": datetime("2011-06-05T03:46:01.000Z"), "friend-ids": {{ 24818185, 42512828, 22798434, 38901116, 12147430, 47942796, 34742031, 7142883, 11882526, 16055416, 3892909, 12824325, 13378363, 34281637, 15457426, 24092146, 27678834, 15804956 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2007-04-12"), "end-date": date("2009-05-09") } ] }
+{ "id": 10132771, "id-copy": 10132771, "alias": "Gaenor", "name": "GaenorEvans", "user-since": datetime("2006-01-23T20:07:34.000Z"), "user-since-copy": datetime("2006-01-23T20:07:34.000Z"), "friend-ids": {{ 20344517, 47988409, 39449785, 16775663, 20200468 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-03-17"), "end-date": null } ] }
+{ "id": 10136659, "id-copy": 10136659, "alias": "Robt", "name": "RobtKooser", "user-since": datetime("2008-11-08T19:22:49.000Z"), "user-since-copy": datetime("2008-11-08T19:22:49.000Z"), "friend-ids": {{ 22245145, 29285750, 9880896 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2001-02-07"), "end-date": null } ] }
+{ "id": 10166767, "id-copy": 10166767, "alias": "Leon", "name": "LeonWardle", "user-since": datetime("2008-05-19T07:05:45.000Z"), "user-since-copy": datetime("2008-05-19T07:05:45.000Z"), "friend-ids": {{ 41883510, 44504996, 36617462, 32609381, 11246739, 18717645, 32225763, 25136144, 18258339, 4951535, 40063362, 38810936, 1994155, 16613514, 25411748, 34221779, 44135463 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2006-10-11"), "end-date": null } ] }
+{ "id": 10173691, "id-copy": 10173691, "alias": "Elissa", "name": "ElissaWilliams", "user-since": datetime("2011-09-26T16:07:17.000Z"), "user-since-copy": datetime("2011-09-26T16:07:17.000Z"), "friend-ids": {{ 2526422 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2001-07-22"), "end-date": null } ] }
+{ "id": 10189600, "id-copy": 10189600, "alias": "Melisa", "name": "MelisaGarry", "user-since": datetime("2010-05-10T10:35:49.000Z"), "user-since-copy": datetime("2010-05-10T10:35:49.000Z"), "friend-ids": {{ 18172527, 26205741, 32077713, 41214698, 33783052, 5734397, 46101468, 30210046, 27425699 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2011-05-20"), "end-date": date("2011-07-20") } ] }
+{ "id": 10202302, "id-copy": 10202302, "alias": "Camila", "name": "CamilaKelley", "user-since": datetime("2010-04-17T06:57:52.000Z"), "user-since-copy": datetime("2010-04-17T06:57:52.000Z"), "friend-ids": {{ 21392718, 41703679, 41044232, 47307848, 13912958, 45329595, 33360889, 24572594, 23726460, 9181899, 42227287, 26565775, 12665691, 12244453, 26966326, 3189268, 41340076, 33904406, 38048631, 22870005 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2000-10-09"), "end-date": null } ] }
+{ "id": 10206877, "id-copy": 10206877, "alias": "Tammie", "name": "TammieBerry", "user-since": datetime("2009-10-14T12:57:11.000Z"), "user-since-copy": datetime("2009-10-14T12:57:11.000Z"), "friend-ids": {{ 23748102, 37944735, 42193629, 11409119, 41246083, 35024235 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2008-05-21"), "end-date": null } ] }
+{ "id": 10250857, "id-copy": 10250857, "alias": "Kandi", "name": "KandiFranks", "user-since": datetime("2010-11-24T19:47:41.000Z"), "user-since-copy": datetime("2010-11-24T19:47:41.000Z"), "friend-ids": {{ 44991748, 27655130, 7925482, 33419150, 18275478 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2002-04-08"), "end-date": null } ] }
+{ "id": 10257028, "id-copy": 10257028, "alias": "Gary", "name": "GaryThompson", "user-since": datetime("2009-01-23T04:15:30.000Z"), "user-since-copy": datetime("2009-01-23T04:15:30.000Z"), "friend-ids": {{ 46006273, 33435458, 40976127, 42353737, 37166855, 14882549, 27357892, 31126471, 38151307, 38721200 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2008-05-03"), "end-date": date("2011-09-08") } ] }
+{ "id": 10258114, "id-copy": 10258114, "alias": "Chuck", "name": "ChuckGibson", "user-since": datetime("2012-07-20T03:48:15.000Z"), "user-since-copy": datetime("2012-07-20T03:48:15.000Z"), "friend-ids": {{ 32318205, 37049120, 26298456, 3281723, 14892306, 29998569, 29992020, 36383932, 15333422, 29670243 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2006-10-11"), "end-date": date("2011-09-02") } ] }
+{ "id": 10272571, "id-copy": 10272571, "alias": "Jarrett", "name": "JarrettGoldvogel", "user-since": datetime("2010-04-28T23:24:22.000Z"), "user-since-copy": datetime("2010-04-28T23:24:22.000Z"), "friend-ids": {{ 47024505, 36647273, 32152567, 28239957, 11739703, 47515825, 17408763, 41224279, 41487670, 43339913 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2004-04-06"), "end-date": date("2010-02-14") } ] }
+{ "id": 10295389, "id-copy": 10295389, "alias": "Major", "name": "MajorDrabble", "user-since": datetime("2009-05-23T12:56:48.000Z"), "user-since-copy": datetime("2009-05-23T12:56:48.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2000-10-26"), "end-date": null } ] }
+{ "id": 10305280, "id-copy": 10305280, "alias": "Isabella", "name": "IsabellaWilo", "user-since": datetime("2007-01-03T11:54:28.000Z"), "user-since-copy": datetime("2007-01-03T11:54:28.000Z"), "friend-ids": {{ 46537100, 26395353, 23044918 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2005-01-06"), "end-date": null } ] }
+{ "id": 10307032, "id-copy": 10307032, "alias": "Quentin", "name": "QuentinSauter", "user-since": datetime("2012-07-11T07:16:43.000Z"), "user-since-copy": datetime("2012-07-11T07:16:43.000Z"), "friend-ids": {{ 1926278, 42211794, 1508832, 14973540, 6721046, 28872485, 5047722, 7805271, 31508326, 20891455, 38735410, 13190567, 18209753, 44468536, 34640135, 47290587, 25576626 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2012-02-13"), "end-date": null } ] }
+{ "id": 10307155, "id-copy": 10307155, "alias": "Rhetta", "name": "RhettaGarneys", "user-since": datetime("2008-03-17T00:33:40.000Z"), "user-since-copy": datetime("2008-03-17T00:33:40.000Z"), "friend-ids": {{ 5658375, 40536479, 47961112, 28517297, 26103231, 32434876, 44285321, 44471686 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2006-06-07"), "end-date": date("2010-10-03") } ] }
+{ "id": 10318882, "id-copy": 10318882, "alias": "Skyler", "name": "SkylerConrad", "user-since": datetime("2007-03-04T08:56:54.000Z"), "user-since-copy": datetime("2007-03-04T08:56:54.000Z"), "friend-ids": {{ 4254240, 3778434, 23914534, 16376376, 39143316, 37229152, 32778982, 30182686, 13077652, 20439638, 34086734, 12101909, 47011547, 28666460, 31034524, 47508299, 17267782, 1260337, 43500601, 914291, 1786773 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2009-01-15"), "end-date": null } ] }
+{ "id": 10320979, "id-copy": 10320979, "alias": "Giuseppe", "name": "GiuseppePorter", "user-since": datetime("2006-10-21T21:56:23.000Z"), "user-since-copy": datetime("2006-10-21T21:56:23.000Z"), "friend-ids": {{ 34102109, 41585396, 8170669, 7376463, 11841426, 6745396, 35637670, 38513040, 26085708, 7577827, 4793535, 31185038, 9126, 502656, 18672743, 27688404, 19846788, 47731814, 42609593 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2012-06-08"), "end-date": null } ] }
+{ "id": 10322398, "id-copy": 10322398, "alias": "Alanna", "name": "AlannaBollinger", "user-since": datetime("2008-09-01T20:05:18.000Z"), "user-since-copy": datetime("2008-09-01T20:05:18.000Z"), "friend-ids": {{ 4294902, 42664964 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2009-10-08"), "end-date": date("2011-09-26") } ] }
+{ "id": 10337950, "id-copy": 10337950, "alias": "Bibi", "name": "BibiCattley", "user-since": datetime("2007-11-16T11:08:34.000Z"), "user-since-copy": datetime("2007-11-16T11:08:34.000Z"), "friend-ids": {{ 24399247, 18391359, 18215808, 36042641, 19360937, 2039633, 17280287, 22159187, 31245932, 4767019, 3299881, 12321916, 22533524, 18760130, 31303729, 39565694, 21606207, 8226305, 16276064 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2003-02-25"), "end-date": date("2008-08-20") } ] }
+{ "id": 10349656, "id-copy": 10349656, "alias": "Woodrow", "name": "WoodrowRichter", "user-since": datetime("2006-09-18T16:22:12.000Z"), "user-since-copy": datetime("2006-09-18T16:22:12.000Z"), "friend-ids": {{ 12344306, 36484394, 30889842, 47572749, 42102868, 22350773, 7166034, 16132372, 45197714, 34516830, 47108654, 4399888, 24401048, 32578065, 16593311, 33394001, 7356357, 29943304, 30866764, 11942891 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2003-11-18"), "end-date": date("2004-10-16") } ] }
+{ "id": 10350421, "id-copy": 10350421, "alias": "Diane", "name": "DianeFisher", "user-since": datetime("2010-10-19T11:08:52.000Z"), "user-since-copy": datetime("2010-10-19T11:08:52.000Z"), "friend-ids": {{ 22455675, 20415125, 21917591, 44414352, 39158851, 3446534, 6627839, 28358200, 1176552, 37914774 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2008-06-04"), "end-date": date("2009-09-11") } ] }
+{ "id": 10365688, "id-copy": 10365688, "alias": "Innocent", "name": "InnocentBlatenberger", "user-since": datetime("2008-11-09T13:57:34.000Z"), "user-since-copy": datetime("2008-11-09T13:57:34.000Z"), "friend-ids": {{ 27902413, 27226238, 35017422, 28154221 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2011-09-13"), "end-date": date("2011-02-05") } ] }
+{ "id": 10391044, "id-copy": 10391044, "alias": "Kendrick", "name": "KendrickNabholz", "user-since": datetime("2007-10-11T19:49:13.000Z"), "user-since-copy": datetime("2007-10-11T19:49:13.000Z"), "friend-ids": {{ 39264696, 35794708, 222108, 29542536, 34470710, 16736694, 36282306, 12411530, 12507843, 30193842, 45764599, 32250152, 16472135, 26507230, 17443301, 16787960, 17651924, 37659951, 28610616, 12928071 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2007-05-07"), "end-date": null } ] }
+{ "id": 10415575, "id-copy": 10415575, "alias": "Amabel", "name": "AmabelRoose", "user-since": datetime("2011-05-28T10:47:28.000Z"), "user-since-copy": datetime("2011-05-28T10:47:28.000Z"), "friend-ids": {{ 22120342, 22881927, 39043768, 27695122, 8669783, 25973892 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2000-03-14"), "end-date": null } ] }
+{ "id": 10451932, "id-copy": 10451932, "alias": "Kory", "name": "KoryRomanoff", "user-since": datetime("2008-09-27T13:29:18.000Z"), "user-since-copy": datetime("2008-09-27T13:29:18.000Z"), "friend-ids": {{ 21328124, 47569968, 22569123, 34316877, 36016117, 19944396, 34862141, 14875173, 3888684, 25235679, 7930355, 24991146, 2862320, 9552488, 23394143, 6292732, 23109993 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2008-04-25"), "end-date": date("2010-03-18") } ] }
+{ "id": 10453144, "id-copy": 10453144, "alias": "Jason", "name": "JasonSachse", "user-since": datetime("2009-01-25T10:27:17.000Z"), "user-since-copy": datetime("2009-01-25T10:27:17.000Z"), "friend-ids": {{ 12949882, 32048809, 23087453, 3994051, 20775019, 22184704, 38106058, 34520240, 13724092, 16309751, 25955640, 4812195, 40546554, 12695295, 16574455, 38615670, 43405164, 7997407, 12239790 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2005-08-01"), "end-date": date("2008-02-08") } ] }
+{ "id": 10458316, "id-copy": 10458316, "alias": "Nivek", "name": "NivekHarper", "user-since": datetime("2009-06-27T16:14:07.000Z"), "user-since-copy": datetime("2009-06-27T16:14:07.000Z"), "friend-ids": {{ 28377255, 40295259, 41434117, 37075748, 12913111, 1533923, 393103, 31161713, 13106373, 924904, 14927212, 7552938, 8299772, 28404911, 45464821, 34440085, 36216015, 2915789, 13470222, 34755494, 29250423 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2002-03-28"), "end-date": date("2010-12-09") } ] }
+{ "id": 10464121, "id-copy": 10464121, "alias": "Enriqueta", "name": "EnriquetaHincken", "user-since": datetime("2005-11-19T09:43:20.000Z"), "user-since-copy": datetime("2005-11-19T09:43:20.000Z"), "friend-ids": {{ 31238269, 29421316, 14426443, 30128291, 9926275, 33523504, 19113054, 402505, 12662005, 36090974, 8733776, 18706660, 14174144, 46009221, 17906304, 41780430, 21807110, 22521282, 21492740, 34033053, 16784027, 11948555 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2008-09-23"), "end-date": null } ] }
+{ "id": 10469980, "id-copy": 10469980, "alias": "Rosalynne", "name": "RosalynneZalack", "user-since": datetime("2012-03-07T10:12:20.000Z"), "user-since-copy": datetime("2012-03-07T10:12:20.000Z"), "friend-ids": {{ 46118617, 27264184, 8045697, 30832992, 47861079, 24266748, 10689886, 14799850, 1178687, 39540720, 17568852, 24394222, 10078451, 4748570, 47808632, 35277954, 8802885, 13747535, 22203533, 42065169, 19096770, 14087466, 45753492 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2006-04-15"), "end-date": date("2010-07-14") } ] }
+{ "id": 10474273, "id-copy": 10474273, "alias": "Juliana", "name": "JulianaLing", "user-since": datetime("2005-05-04T20:58:12.000Z"), "user-since-copy": datetime("2005-05-04T20:58:12.000Z"), "friend-ids": {{ 8881381, 34113161, 15553599, 40081858, 12450920, 42147178, 568875, 11891228, 13309462, 39127120, 34765111, 19162279, 29505162, 891909, 33485893, 25658561, 36146447, 37027867, 39396759 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2008-03-03"), "end-date": null } ] }
+{ "id": 10486213, "id-copy": 10486213, "alias": "Modesto", "name": "ModestoCox", "user-since": datetime("2006-02-07T05:43:24.000Z"), "user-since-copy": datetime("2006-02-07T05:43:24.000Z"), "friend-ids": {{ 42665859, 12929499, 5618502, 24287766, 38722882, 5162913, 2978226, 37521984, 43144325, 3313029, 17680751, 726799 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2007-08-12"), "end-date": null } ] }
+{ "id": 10487029, "id-copy": 10487029, "alias": "Fredericka", "name": "FrederickaShea", "user-since": datetime("2011-04-07T06:12:40.000Z"), "user-since-copy": datetime("2011-04-07T06:12:40.000Z"), "friend-ids": {{ 45223639, 1019151, 30626857, 10247171, 36952244, 36646177, 2396690, 26604216, 19215860, 20900949, 14160764 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2012-01-08"), "end-date": null } ] }
+{ "id": 10493269, "id-copy": 10493269, "alias": "Anya", "name": "AnyaWoodward", "user-since": datetime("2009-03-08T07:08:04.000Z"), "user-since-copy": datetime("2009-03-08T07:08:04.000Z"), "friend-ids": {{ 2357333 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2001-05-04"), "end-date": null } ] }
+{ "id": 10503262, "id-copy": 10503262, "alias": "Suzanne", "name": "SuzanneFonblanque", "user-since": datetime("2012-03-16T20:22:06.000Z"), "user-since-copy": datetime("2012-03-16T20:22:06.000Z"), "friend-ids": {{ 17868500, 500991, 7701699, 45401842, 16746916, 24217608, 46250003, 17567888, 28186634 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2001-01-18"), "end-date": date("2005-08-07") } ] }
+{ "id": 10508467, "id-copy": 10508467, "alias": "Quincey", "name": "QuinceyKettlewell", "user-since": datetime("2009-11-08T14:09:57.000Z"), "user-since-copy": datetime("2009-11-08T14:09:57.000Z"), "friend-ids": {{ 16037923, 33757766, 22829568, 34589661, 10645853, 43124745, 41785968, 27704416, 42381402, 11993654, 31993782, 37761743, 15571469, 33326934, 22719288, 18321279, 19252211, 42927515, 22390312, 37655021, 37511969, 47740024, 1015876 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2009-02-24"), "end-date": null } ] }
+{ "id": 10509676, "id-copy": 10509676, "alias": "Dinorah", "name": "DinorahRopes", "user-since": datetime("2009-12-05T06:00:03.000Z"), "user-since-copy": datetime("2009-12-05T06:00:03.000Z"), "friend-ids": {{ 13297859, 17139775, 6500776, 46867326, 18510471, 20417055, 39500392, 2482383, 3361807, 14184772, 24928547, 14390842, 40519232, 14991589, 21242930, 24964529, 38160860, 25523267, 4709228, 13473948, 15850888, 30150938, 5984402, 26343874 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2010-08-11"), "end-date": null } ] }
+{ "id": 10513507, "id-copy": 10513507, "alias": "Jasmin", "name": "JasminHatfield", "user-since": datetime("2009-06-25T22:45:16.000Z"), "user-since-copy": datetime("2009-06-25T22:45:16.000Z"), "friend-ids": {{ 31323261 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2000-05-12"), "end-date": date("2003-05-07") } ] }
+{ "id": 10529809, "id-copy": 10529809, "alias": "Aric", "name": "AricLauffer", "user-since": datetime("2007-05-18T09:08:29.000Z"), "user-since-copy": datetime("2007-05-18T09:08:29.000Z"), "friend-ids": {{ 36647795, 13183862, 5313167, 36450019, 46412788, 47789981, 4012027, 35872968, 3903895 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2011-09-22"), "end-date": null } ] }
+{ "id": 10532791, "id-copy": 10532791, "alias": "Byrne", "name": "ByrneLafortune", "user-since": datetime("2010-03-13T13:21:05.000Z"), "user-since-copy": datetime("2010-03-13T13:21:05.000Z"), "friend-ids": {{ 35020297, 40002497, 16857157, 47134232, 37864297, 31029450, 36968713, 36672267, 15503365, 43888732, 29395734, 35372186, 19093208, 21774877, 9785166, 22833579 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2003-12-21"), "end-date": null } ] }
+{ "id": 10533343, "id-copy": 10533343, "alias": "Gwendolen", "name": "GwendolenHanseu", "user-since": datetime("2007-02-04T19:56:51.000Z"), "user-since-copy": datetime("2007-02-04T19:56:51.000Z"), "friend-ids": {{ 25281794, 21814505, 11684475, 5599252, 17261378, 11061422, 27392332, 47872606, 39198697, 17314413, 4034634, 42776559, 43885593, 24835625, 18150148, 4946129, 9288372, 5675162, 34976580 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2010-07-10"), "end-date": null } ] }
+{ "id": 10540441, "id-copy": 10540441, "alias": "Albert", "name": "AlbertBasinger", "user-since": datetime("2007-05-12T06:03:38.000Z"), "user-since-copy": datetime("2007-05-12T06:03:38.000Z"), "friend-ids": {{ 36392592, 35815177, 22050314, 45279196, 15405747, 33802667, 44081359, 2027267, 47159697, 20007080 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2002-10-04"), "end-date": date("2005-08-17") } ] }
+{ "id": 10540825, "id-copy": 10540825, "alias": "Jayna", "name": "JaynaRowe", "user-since": datetime("2008-01-09T12:09:19.000Z"), "user-since-copy": datetime("2008-01-09T12:09:19.000Z"), "friend-ids": {{ 20315422, 9358699, 6204561, 40594838, 46678685, 34224970, 47262413, 42477325, 7591560, 39986319, 9438124, 30292072, 11187685, 27885, 47428887, 9535830, 36979072, 14613793 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2005-11-10"), "end-date": null } ] }
+{ "id": 10552405, "id-copy": 10552405, "alias": "Les", "name": "LesBarth", "user-since": datetime("2008-04-02T11:02:37.000Z"), "user-since-copy": datetime("2008-04-02T11:02:37.000Z"), "friend-ids": {{ 33645432, 43039707 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2000-09-18"), "end-date": null } ] }
+{ "id": 10563310, "id-copy": 10563310, "alias": "Justina", "name": "JustinaHall", "user-since": datetime("2010-08-24T08:57:45.000Z"), "user-since-copy": datetime("2010-08-24T08:57:45.000Z"), "friend-ids": {{ 42796179, 25994871, 35439919, 28722419, 7189994, 41505357, 35095639, 14693797, 36519323, 32598167, 6323551, 14565174, 35997662, 9705559, 3996730 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2011-02-20"), "end-date": date("2011-05-05") } ] }
+{ "id": 10573795, "id-copy": 10573795, "alias": "Neil", "name": "NeilMilne", "user-since": datetime("2005-11-15T02:57:46.000Z"), "user-since-copy": datetime("2005-11-15T02:57:46.000Z"), "friend-ids": {{ 33469327, 4261514, 43412669, 17289131, 27535421, 15267017, 14005060 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2001-11-13"), "end-date": date("2001-10-28") } ] }
+{ "id": 10577128, "id-copy": 10577128, "alias": "Charnette", "name": "CharnettePyle", "user-since": datetime("2008-08-20T21:25:22.000Z"), "user-since-copy": datetime("2008-08-20T21:25:22.000Z"), "friend-ids": {{ 30078840, 16315930, 12006652, 31984600, 12053254, 41773411, 43318427, 21592935, 40739515, 30608076, 21922300, 5687640 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2001-11-25"), "end-date": date("2002-08-12") } ] }
+{ "id": 10579345, "id-copy": 10579345, "alias": "Rexana", "name": "RexanaSchaeffer", "user-since": datetime("2006-01-20T15:37:57.000Z"), "user-since-copy": datetime("2006-01-20T15:37:57.000Z"), "friend-ids": {{ 20070497, 44547094, 38571608, 30731404, 7825730, 8433351, 25090042, 38943273, 3599029, 28517891, 17427828, 6853394, 32856065, 46627870, 43885788 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2012-01-22"), "end-date": null } ] }
+{ "id": 10580422, "id-copy": 10580422, "alias": "Travers", "name": "TraversSadley", "user-since": datetime("2011-02-09T08:22:49.000Z"), "user-since-copy": datetime("2011-02-09T08:22:49.000Z"), "friend-ids": {{ 36067992, 8651663, 43180149, 732576, 35709545, 30999437 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2009-05-27"), "end-date": null } ] }
+{ "id": 10591498, "id-copy": 10591498, "alias": "Mick", "name": "MickVeith", "user-since": datetime("2006-02-21T06:58:53.000Z"), "user-since-copy": datetime("2006-02-21T06:58:53.000Z"), "friend-ids": {{ 33872347, 40692511, 18563650 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2011-08-07"), "end-date": date("2011-01-10") } ] }
+{ "id": 10595164, "id-copy": 10595164, "alias": "Jerome", "name": "JeromeLacon", "user-since": datetime("2009-09-24T09:47:36.000Z"), "user-since-copy": datetime("2009-09-24T09:47:36.000Z"), "friend-ids": {{ 31538601 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2012-07-26"), "end-date": null } ] }
+{ "id": 10610356, "id-copy": 10610356, "alias": "Jason", "name": "JasonGearhart", "user-since": datetime("2010-03-05T22:57:20.000Z"), "user-since-copy": datetime("2010-03-05T22:57:20.000Z"), "friend-ids": {{ 6967239, 47468231, 29517365, 9206260 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2012-03-16"), "end-date": date("2012-06-19") } ] }
+{ "id": 10623790, "id-copy": 10623790, "alias": "Leon", "name": "LeonSouthern", "user-since": datetime("2006-08-26T12:47:17.000Z"), "user-since-copy": datetime("2006-08-26T12:47:17.000Z"), "friend-ids": {{ 15974929, 10054172, 9775689, 22060162, 41777649, 13548836, 10842789, 45455670, 32027368, 45268626, 40570545, 18214851, 47559589, 38267347, 41101925, 45749689, 29277572, 47828706, 45708476, 33769625 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2012-06-22"), "end-date": date("2012-06-05") } ] }
+{ "id": 10635319, "id-copy": 10635319, "alias": "Rusty", "name": "RustyStange", "user-since": datetime("2010-08-17T17:30:37.000Z"), "user-since-copy": datetime("2010-08-17T17:30:37.000Z"), "friend-ids": {{ 28180565, 25608756 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2006-07-07"), "end-date": null } ] }
+{ "id": 10637896, "id-copy": 10637896, "alias": "Hiram", "name": "HiramRohtin", "user-since": datetime("2006-11-05T14:44:03.000Z"), "user-since-copy": datetime("2006-11-05T14:44:03.000Z"), "friend-ids": {{ 1387663, 11367203, 24828245 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2012-02-18"), "end-date": date("2012-02-12") } ] }
+{ "id": 10650265, "id-copy": 10650265, "alias": "Kristia", "name": "KristiaCowart", "user-since": datetime("2005-09-27T20:13:12.000Z"), "user-since-copy": datetime("2005-09-27T20:13:12.000Z"), "friend-ids": {{ 41553475, 45442923, 20846576, 6432869, 40830841 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2006-05-05"), "end-date": null } ] }
+{ "id": 10658977, "id-copy": 10658977, "alias": "Danny", "name": "DannyBailey", "user-since": datetime("2006-12-12T12:28:17.000Z"), "user-since-copy": datetime("2006-12-12T12:28:17.000Z"), "friend-ids": {{ 27744791, 5839976, 37243832, 42061553, 15660549, 26723434, 25864049, 8038100, 47690286, 29206337, 6169296, 1933137, 6500848, 45632949, 6329147, 15602171, 13477556, 25033716, 9515038, 4081408, 42840830 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2009-12-14"), "end-date": date("2009-03-11") } ] }
+{ "id": 10668283, "id-copy": 10668283, "alias": "Dorian", "name": "DorianTomlinson", "user-since": datetime("2008-06-22T00:01:46.000Z"), "user-since-copy": datetime("2008-06-22T00:01:46.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2004-12-17"), "end-date": null } ] }
+{ "id": 10686646, "id-copy": 10686646, "alias": "Deborah", "name": "DeborahRosenstiehl", "user-since": datetime("2012-06-18T16:51:32.000Z"), "user-since-copy": datetime("2012-06-18T16:51:32.000Z"), "friend-ids": {{ 34005621, 6910583, 11226890, 1333457, 13615971, 15332838, 30484423, 38261521, 39526604, 12093262, 15397660, 29644860, 36715060, 16753181 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2005-07-01"), "end-date": date("2007-10-22") } ] }
+{ "id": 10690066, "id-copy": 10690066, "alias": "Abraham", "name": "AbrahamWardle", "user-since": datetime("2006-04-08T20:27:10.000Z"), "user-since-copy": datetime("2006-04-08T20:27:10.000Z"), "friend-ids": {{ 18105973, 39839261, 27532181, 2565949, 37077592, 28929530 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2005-06-03"), "end-date": date("2006-12-02") } ] }
+{ "id": 10700431, "id-copy": 10700431, "alias": "Lessie", "name": "LessieRobinson", "user-since": datetime("2011-02-03T18:31:41.000Z"), "user-since-copy": datetime("2011-02-03T18:31:41.000Z"), "friend-ids": {{ 8174251, 46379649, 3507858, 13269282, 38334885, 12074283, 34128956, 46802811, 37285621, 15203773, 17611824, 47823053, 28609781, 31377970, 11077457, 3771375, 27529933, 170454, 38682017 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2002-01-08"), "end-date": date("2006-06-08") } ] }
+{ "id": 10710526, "id-copy": 10710526, "alias": "Heike", "name": "HeikeReed", "user-since": datetime("2009-08-15T19:20:30.000Z"), "user-since-copy": datetime("2009-08-15T19:20:30.000Z"), "friend-ids": {{ 36253853, 35694929, 43324582, 24829816 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2011-02-12"), "end-date": date("2011-01-22") } ] }
+{ "id": 10714447, "id-copy": 10714447, "alias": "Leone", "name": "LeoneCoughenour", "user-since": datetime("2012-06-13T05:05:11.000Z"), "user-since-copy": datetime("2012-06-13T05:05:11.000Z"), "friend-ids": {{ 13098839, 21185838, 26566436, 37464340, 8086775, 37143068, 40377316, 39371296 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2005-04-16"), "end-date": null } ] }
+{ "id": 10729942, "id-copy": 10729942, "alias": "Valda", "name": "ValdaFea", "user-since": datetime("2005-07-16T09:31:53.000Z"), "user-since-copy": datetime("2005-07-16T09:31:53.000Z"), "friend-ids": {{ 20145015, 42027050, 38819467, 3406065, 4977132, 47154979, 23685067 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2008-10-12"), "end-date": null } ] }
+{ "id": 10735369, "id-copy": 10735369, "alias": "Cody", "name": "CodySchaeffer", "user-since": datetime("2008-07-03T05:27:24.000Z"), "user-since-copy": datetime("2008-07-03T05:27:24.000Z"), "friend-ids": {{ 15534779, 12333665, 10468027, 3865324, 39537208, 16999101, 9009757, 318331, 30123714, 10137427, 16481424 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2009-01-19"), "end-date": null } ] }
+{ "id": 10777441, "id-copy": 10777441, "alias": "Rosaline", "name": "RosalineFaast", "user-since": datetime("2005-05-23T08:24:59.000Z"), "user-since-copy": datetime("2005-05-23T08:24:59.000Z"), "friend-ids": {{ 25088415, 36453219, 42450810, 6845863, 23568088, 34305276, 28849557, 41593223, 18542045, 37652004, 9159129, 42079452 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2002-01-04"), "end-date": null } ] }
+{ "id": 10786129, "id-copy": 10786129, "alias": "Ardelle", "name": "ArdelleHoopengarner", "user-since": datetime("2012-05-27T08:36:37.000Z"), "user-since-copy": datetime("2012-05-27T08:36:37.000Z"), "friend-ids": {{ 44854493, 13697746, 8918104, 22353878, 46059542, 23393155, 37374548, 1531344, 31554501, 30390740, 10076243, 19028830, 46174212, 4991316, 30988902, 6717568 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2011-03-08"), "end-date": null } ] }
+{ "id": 10789207, "id-copy": 10789207, "alias": "Lucinda", "name": "LucindaFillmore", "user-since": datetime("2009-11-13T18:35:41.000Z"), "user-since-copy": datetime("2009-11-13T18:35:41.000Z"), "friend-ids": {{ 10917581, 24902161, 29393856, 35293349, 31477965, 44139676, 18083704, 46487557 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2005-11-04"), "end-date": null } ] }
+{ "id": 10803184, "id-copy": 10803184, "alias": "Daria", "name": "DariaPyle", "user-since": datetime("2010-11-22T05:29:27.000Z"), "user-since-copy": datetime("2010-11-22T05:29:27.000Z"), "friend-ids": {{ 26747755, 39431389, 24370112, 37832812, 20626868, 30614988, 38041392, 31908762, 47561829, 45121087, 24496373, 32944554, 16470795, 11915899, 29900938, 4003497, 38829225, 36390033, 36474051 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2009-10-03"), "end-date": null } ] }
+{ "id": 10834579, "id-copy": 10834579, "alias": "Penni", "name": "PenniBlunt", "user-since": datetime("2010-05-20T20:29:16.000Z"), "user-since-copy": datetime("2010-05-20T20:29:16.000Z"), "friend-ids": {{ 25926886, 10263270, 4098530, 40765625, 16591278 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2003-03-22"), "end-date": null } ] }
+{ "id": 10836430, "id-copy": 10836430, "alias": "Kaycee", "name": "KayceeCatleay", "user-since": datetime("2007-05-18T07:19:02.000Z"), "user-since-copy": datetime("2007-05-18T07:19:02.000Z"), "friend-ids": {{ 40568633, 44667158, 18923311, 34987631, 29306332, 38711535, 43999451, 3179954, 9799980, 3451381, 23204288, 17797804, 2164448, 16697308, 24697554, 45250786, 10029328, 27871642 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2002-04-25"), "end-date": null } ] }
+{ "id": 10837876, "id-copy": 10837876, "alias": "Tianna", "name": "TiannaOppenheimer", "user-since": datetime("2006-05-14T01:19:23.000Z"), "user-since-copy": datetime("2006-05-14T01:19:23.000Z"), "friend-ids": {{ 8389212, 20540523, 37708985, 22298925, 5938365, 34705514, 39174355, 44283530, 44597508, 37912034, 45434053, 47086440, 6559664, 12451920, 47639456, 39030619, 24239344, 2566247, 27102794 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2008-04-03"), "end-date": null } ] }
+{ "id": 10840990, "id-copy": 10840990, "alias": "Libby", "name": "LibbyHayhurst", "user-since": datetime("2009-10-28T22:52:04.000Z"), "user-since-copy": datetime("2009-10-28T22:52:04.000Z"), "friend-ids": {{ 32146321, 47850956, 42432761, 28856789, 18595962, 23408710, 37015546 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2003-01-20"), "end-date": null } ] }
+{ "id": 10858339, "id-copy": 10858339, "alias": "Eugenio", "name": "EugenioLangston", "user-since": datetime("2006-06-14T22:24:18.000Z"), "user-since-copy": datetime("2006-06-14T22:24:18.000Z"), "friend-ids": {{ 18107191, 19162062, 26048227, 16199255, 32644324, 3917262, 38994370, 36221435, 34919041 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2011-02-03"), "end-date": null } ] }
+{ "id": 10869727, "id-copy": 10869727, "alias": "Jacquetta", "name": "JacquettaMaugham", "user-since": datetime("2010-07-11T22:43:19.000Z"), "user-since-copy": datetime("2010-07-11T22:43:19.000Z"), "friend-ids": {{ 36109878, 46889968, 19648550, 14051620, 14645938, 14933447, 33880415 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2002-12-09"), "end-date": null } ] }
+{ "id": 10888777, "id-copy": 10888777, "alias": "Bevis", "name": "BevisStall", "user-since": datetime("2007-04-05T02:35:27.000Z"), "user-since-copy": datetime("2007-04-05T02:35:27.000Z"), "friend-ids": {{ 1924847, 33036971, 5163765, 37816368, 15975671, 11388174, 38485519, 43186487, 30402693, 34350975, 24348537, 34349089, 22680019, 30625064, 23751465, 9072515, 15915109 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2005-03-11"), "end-date": null } ] }
+{ "id": 10902049, "id-copy": 10902049, "alias": "Fae", "name": "FaeRing", "user-since": datetime("2008-06-15T12:54:57.000Z"), "user-since-copy": datetime("2008-06-15T12:54:57.000Z"), "friend-ids": {{ 2667467, 46445373, 11696423, 42003744, 47667382, 34088774, 4279683, 29934858, 21213543, 44195034, 38786294, 14946433, 38805114, 9972575, 3309290, 5324029, 32663319, 20577589, 9110909, 27272396, 47622938 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2007-11-15"), "end-date": null } ] }
+{ "id": 10905721, "id-copy": 10905721, "alias": "Tibby", "name": "TibbyPriebe", "user-since": datetime("2010-04-09T18:32:02.000Z"), "user-since-copy": datetime("2010-04-09T18:32:02.000Z"), "friend-ids": {{ 18406663, 1072532, 16897765 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2007-05-05"), "end-date": date("2007-03-06") } ] }
+{ "id": 10907953, "id-copy": 10907953, "alias": "Wymond", "name": "WymondSnyder", "user-since": datetime("2006-02-25T03:33:22.000Z"), "user-since-copy": datetime("2006-02-25T03:33:22.000Z"), "friend-ids": {{ 16280602, 26846293, 39235173, 4686537, 30457440, 23649561, 34348317, 28099021, 1622222, 24073647, 4742953, 14925763, 17026705, 46257859, 22592244 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2012-07-22"), "end-date": null } ] }
+{ "id": 10913971, "id-copy": 10913971, "alias": "Marylyn", "name": "MarylynBuehler", "user-since": datetime("2008-03-02T11:14:18.000Z"), "user-since-copy": datetime("2008-03-02T11:14:18.000Z"), "friend-ids": {{ 36555710, 21041383, 37895483, 11392039, 5195346, 12022072, 5206222, 37834919, 434970, 4441054, 39212196, 12773393 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2000-01-11"), "end-date": null } ] }
+{ "id": 10948315, "id-copy": 10948315, "alias": "Munro", "name": "MunroDiegel", "user-since": datetime("2006-11-24T10:55:36.000Z"), "user-since-copy": datetime("2006-11-24T10:55:36.000Z"), "friend-ids": {{ 46912879, 47760999, 8438850, 12005776, 7286415, 41598308, 42462653, 2040525, 8432844, 39644931 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2011-08-08"), "end-date": date("2011-09-27") } ] }
+{ "id": 10975810, "id-copy": 10975810, "alias": "Davin", "name": "DavinKifer", "user-since": datetime("2005-08-19T20:23:07.000Z"), "user-since-copy": datetime("2005-08-19T20:23:07.000Z"), "friend-ids": {{ 20162027, 7842505, 3191764, 11487126, 44589086, 14959953, 18826364, 18917713, 37717273, 24319173, 1393081, 19608709, 47932966, 37681921, 47734310, 21616345, 21035793, 9650227, 43642280, 21890130, 17249802, 27944839 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2011-12-17"), "end-date": date("2011-12-01") } ] }
+{ "id": 10985830, "id-copy": 10985830, "alias": "Spencer", "name": "SpencerWilo", "user-since": datetime("2010-03-02T07:41:59.000Z"), "user-since-copy": datetime("2010-03-02T07:41:59.000Z"), "friend-ids": {{ 5766878, 20551454, 27297902, 44757901, 7660518, 28072828, 6387548, 6276027, 40692560, 36168648, 24514885, 40791549, 15536640, 23757967, 19875372 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2004-04-14"), "end-date": date("2009-02-17") } ] }
+{ "id": 11007700, "id-copy": 11007700, "alias": "Elly", "name": "EllyWard", "user-since": datetime("2009-04-20T08:46:09.000Z"), "user-since-copy": datetime("2009-04-20T08:46:09.000Z"), "friend-ids": {{ 9712756, 6523354 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2007-12-07"), "end-date": date("2007-07-27") } ] }
+{ "id": 11015908, "id-copy": 11015908, "alias": "Giuseppe", "name": "GiuseppeWard", "user-since": datetime("2008-09-14T16:37:40.000Z"), "user-since-copy": datetime("2008-09-14T16:37:40.000Z"), "friend-ids": {{ 9972151, 40271551, 46207899, 29987388, 19876511, 47546614, 17051350, 1579198, 2151480, 26507940, 18177808, 25866392, 40253780 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2008-02-24"), "end-date": null } ] }
+{ "id": 11022889, "id-copy": 11022889, "alias": "Aubrey", "name": "AubreyMccallum", "user-since": datetime("2009-08-17T02:42:54.000Z"), "user-since-copy": datetime("2009-08-17T02:42:54.000Z"), "friend-ids": {{ 22265320, 4304911, 3403321, 20791603, 31499855, 22278594, 14580040, 31651270, 14509751, 13733306, 10947101, 7713960 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2001-01-11"), "end-date": null } ] }
+{ "id": 11032186, "id-copy": 11032186, "alias": "Tabby", "name": "TabbySealis", "user-since": datetime("2007-12-10T21:45:46.000Z"), "user-since-copy": datetime("2007-12-10T21:45:46.000Z"), "friend-ids": {{ 8190058, 5089537, 18167034, 19113649, 38817127, 7644664, 12427817, 39615196, 11451538, 27188211, 27425673, 33084974, 10726858, 40696324, 41487982, 42282364, 17084607, 41647211, 40268195, 29075837, 41802984, 9719771, 29747340, 28103359 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2008-07-13"), "end-date": date("2010-12-04") } ] }
+{ "id": 11032477, "id-copy": 11032477, "alias": "Wilmer", "name": "WilmerWortman", "user-since": datetime("2007-06-03T19:27:24.000Z"), "user-since-copy": datetime("2007-06-03T19:27:24.000Z"), "friend-ids": {{ 18685187, 2599612, 27305395, 20825021, 20327586, 21301262, 29222955, 20377452, 11211553, 37446807, 20533832, 10098143, 43828837, 37254072, 46029810, 16401947, 7537056, 41738273, 4665729, 27400110, 146251, 14185116 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2006-03-17"), "end-date": date("2011-08-03") } ] }
+{ "id": 11049715, "id-copy": 11049715, "alias": "Carlo", "name": "CarloBrooks", "user-since": datetime("2005-03-23T21:46:06.000Z"), "user-since-copy": datetime("2005-03-23T21:46:06.000Z"), "friend-ids": {{ 8214850, 7465603, 15385071, 32299168, 5993026, 3262895, 24995417, 25987462, 10230501, 12537459, 44597291, 33492282, 30758369, 15589085, 6799067, 23023304, 42597416, 10978280, 40668626, 25650335, 37336071 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2011-09-15"), "end-date": date("2011-09-03") } ] }
+{ "id": 11051014, "id-copy": 11051014, "alias": "Tad", "name": "TadWilson", "user-since": datetime("2011-05-05T14:48:34.000Z"), "user-since-copy": datetime("2011-05-05T14:48:34.000Z"), "friend-ids": {{ 42862096, 17517240, 8058482, 9927174, 4207109, 4924943, 11531213 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2005-01-25"), "end-date": date("2010-11-14") } ] }
+{ "id": 11052748, "id-copy": 11052748, "alias": "Andriana", "name": "AndrianaYonkie", "user-since": datetime("2005-05-08T19:49:03.000Z"), "user-since-copy": datetime("2005-05-08T19:49:03.000Z"), "friend-ids": {{ 24372868, 41932219, 14088659, 33215970, 34384197, 16343164, 24230672, 20937997, 23129922, 33184913, 25421373, 12081379, 289577, 19330874, 31625333, 34885607, 34353478, 17694263, 34819024, 44837603 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2003-06-16"), "end-date": date("2008-02-15") } ] }
+{ "id": 11061631, "id-copy": 11061631, "alias": "Maxene", "name": "MaxeneKellogg", "user-since": datetime("2005-11-13T01:09:31.000Z"), "user-since-copy": datetime("2005-11-13T01:09:31.000Z"), "friend-ids": {{ 31578394, 39466620, 35741359, 14244925, 3000582, 39031643, 5008430, 18315325, 30440631, 37868108, 12014032, 32314102, 42887702, 1853960, 28022174, 2024670, 38864358, 42073112, 16259942, 34693959, 25315399, 37475597, 33599283 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2008-05-13"), "end-date": null } ] }
+{ "id": 11092324, "id-copy": 11092324, "alias": "Paul", "name": "PaulOneal", "user-since": datetime("2006-11-20T10:50:19.000Z"), "user-since-copy": datetime("2006-11-20T10:50:19.000Z"), "friend-ids": {{ 44707820, 20249424, 18862268, 32895394, 29899430 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2003-01-06"), "end-date": null } ] }
+{ "id": 11103856, "id-copy": 11103856, "alias": "Dennise", "name": "DenniseGarland", "user-since": datetime("2008-10-19T11:09:14.000Z"), "user-since-copy": datetime("2008-10-19T11:09:14.000Z"), "friend-ids": {{ 2613052, 4777379, 29911213, 30822813, 44182985, 803163, 32630608, 7433428, 43625503, 19274272, 20950244, 21434389, 44059623, 40416129, 47937344, 12392360 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2005-04-10"), "end-date": date("2005-07-26") } ] }
+{ "id": 11113168, "id-copy": 11113168, "alias": "Daphne", "name": "DaphneHindman", "user-since": datetime("2011-11-09T02:55:42.000Z"), "user-since-copy": datetime("2011-11-09T02:55:42.000Z"), "friend-ids": {{ 194785, 11696942, 23072861, 37052204, 17574763, 14099428, 44155581 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2002-07-16"), "end-date": date("2006-11-08") } ] }
+{ "id": 11116465, "id-copy": 11116465, "alias": "Read", "name": "ReadOppenheimer", "user-since": datetime("2012-08-23T03:38:20.000Z"), "user-since-copy": datetime("2012-08-23T03:38:20.000Z"), "friend-ids": {{ 18679034, 12828526, 13510152, 28052139, 20367021, 30392195, 41580515, 2644015, 29573423, 22838698 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2009-11-05"), "end-date": null } ] }
+{ "id": 11116594, "id-copy": 11116594, "alias": "Norwood", "name": "NorwoodErrett", "user-since": datetime("2008-10-04T16:36:27.000Z"), "user-since-copy": datetime("2008-10-04T16:36:27.000Z"), "friend-ids": {{ 30996403, 30788997, 22512789, 35425088, 12096858, 21391496, 41281428, 15854003, 47041757, 31205204, 36849089, 43015828, 27098245, 46735331, 9520980, 34482257, 36898055, 8962397 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2002-05-20"), "end-date": null } ] }
+{ "id": 11117371, "id-copy": 11117371, "alias": "Jules", "name": "JulesRichardson", "user-since": datetime("2009-12-06T06:21:58.000Z"), "user-since-copy": datetime("2009-12-06T06:21:58.000Z"), "friend-ids": {{ 75701, 18653454, 5088871, 20583891, 46460448, 19742484, 2433030, 30869605, 9273775, 6556358 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2001-09-17"), "end-date": date("2006-06-05") } ] }
+{ "id": 11129635, "id-copy": 11129635, "alias": "Porter", "name": "PorterRohtin", "user-since": datetime("2005-08-07T05:18:16.000Z"), "user-since-copy": datetime("2005-08-07T05:18:16.000Z"), "friend-ids": {{ 15192554, 37509296, 35638203, 5517199, 3781940, 43497242, 28477558, 4325184, 34919156, 18037278, 36486191, 13966437, 16629611, 40623060 }}, "employment": [ { "organization-name": "Zimcone", "start-date": date("2005-07-13"), "end-date": null } ] }
+{ "id": 11147050, "id-copy": 11147050, "alias": "Karena", "name": "KarenaTanner", "user-since": datetime("2007-03-17T08:50:48.000Z"), "user-since-copy": datetime("2007-03-17T08:50:48.000Z"), "friend-ids": {{ 39952587, 2518830, 30305705, 21365609, 45914603, 2590495, 8595660 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2000-11-13"), "end-date": date("2009-01-10") } ] }
+{ "id": 11152162, "id-copy": 11152162, "alias": "Tennille", "name": "TennilleGongaware", "user-since": datetime("2008-12-22T17:22:19.000Z"), "user-since-copy": datetime("2008-12-22T17:22:19.000Z"), "friend-ids": {{ 38167013, 48016045, 45757020, 26256748, 14740496, 36818162, 43284365, 29637839, 30820213, 535748, 31611626 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2005-07-27"), "end-date": null } ] }
+{ "id": 11203174, "id-copy": 11203174, "alias": "Lise", "name": "LiseRockwell", "user-since": datetime("2005-04-21T02:17:33.000Z"), "user-since-copy": datetime("2005-04-21T02:17:33.000Z"), "friend-ids": {{ 25322984, 687106, 15193641, 24397137, 34772763, 24725595, 30853266, 14933558, 36895249, 39451299, 2620397, 44594032, 3455415, 39921033, 21621070, 800967 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2006-01-13"), "end-date": date("2008-07-23") } ] }
+{ "id": 11209297, "id-copy": 11209297, "alias": "Merlin", "name": "MerlinLambert", "user-since": datetime("2012-07-01T09:30:07.000Z"), "user-since-copy": datetime("2012-07-01T09:30:07.000Z"), "friend-ids": {{ 28451212, 22119974, 1386726, 20860479, 37160852, 38281524, 17165711, 41076637, 19118162 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2012-06-26"), "end-date": date("2012-06-09") } ] }
+{ "id": 11214976, "id-copy": 11214976, "alias": "Maxwell", "name": "MaxwellBailey", "user-since": datetime("2005-11-25T15:01:26.000Z"), "user-since-copy": datetime("2005-11-25T15:01:26.000Z"), "friend-ids": {{ 22027101, 5782023, 46909646, 27593651, 31079804, 31989634, 7337526, 34757530, 32792041 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2011-04-04"), "end-date": null } ] }
+{ "id": 11226055, "id-copy": 11226055, "alias": "Tony", "name": "TonyBowman", "user-since": datetime("2011-06-27T19:37:38.000Z"), "user-since-copy": datetime("2011-06-27T19:37:38.000Z"), "friend-ids": {{ 38143523, 845148, 17273955, 5476646, 28032520, 29082922, 26004648, 7037738, 34413190, 22897549, 19873990, 22338498, 10902206, 43469888, 21968875, 5127825, 11962760, 43764181, 20623302, 23901531, 3402018, 15386752, 30847912, 205201 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2004-09-13"), "end-date": date("2011-01-10") } ] }
+{ "id": 11230663, "id-copy": 11230663, "alias": "Caryl", "name": "CarylSmail", "user-since": datetime("2006-03-17T16:52:51.000Z"), "user-since-copy": datetime("2006-03-17T16:52:51.000Z"), "friend-ids": {{ 32153460, 21186863, 24199212, 25220508, 26590053, 42433121, 35372685 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2007-12-05"), "end-date": null } ] }
+{ "id": 11235340, "id-copy": 11235340, "alias": "Maurice", "name": "MauriceHayhurst", "user-since": datetime("2008-12-24T05:11:37.000Z"), "user-since-copy": datetime("2008-12-24T05:11:37.000Z"), "friend-ids": {{ 36045307, 37144109, 37142113, 38379399, 21011762, 30698208, 3185430, 24698099, 39750599, 1820110, 19740583, 5658727, 33165497, 27066109, 20299488, 26484094, 17984991, 9623240, 15287433, 32468842, 34023148, 16744372, 30389952, 40305465 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2011-05-19"), "end-date": date("2011-11-15") } ] }
+{ "id": 11262439, "id-copy": 11262439, "alias": "Alexandra", "name": "AlexandraStocker", "user-since": datetime("2010-08-28T03:48:52.000Z"), "user-since-copy": datetime("2010-08-28T03:48:52.000Z"), "friend-ids": {{ 16331707 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2010-12-04"), "end-date": null } ] }
+{ "id": 11271517, "id-copy": 11271517, "alias": "Amaryllis", "name": "AmaryllisNewlove", "user-since": datetime("2009-06-10T04:18:11.000Z"), "user-since-copy": datetime("2009-06-10T04:18:11.000Z"), "friend-ids": {{ 6594489, 17958014, 4087759, 38993546, 1741537, 8374107, 30133658, 33873746 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2004-12-21"), "end-date": date("2011-08-19") } ] }
+{ "id": 11272591, "id-copy": 11272591, "alias": "Caris", "name": "CarisCatleay", "user-since": datetime("2007-01-27T07:35:12.000Z"), "user-since-copy": datetime("2007-01-27T07:35:12.000Z"), "friend-ids": {{ 26014944 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2012-07-15"), "end-date": date("2012-07-01") } ] }
+{ "id": 11273587, "id-copy": 11273587, "alias": "Timmy", "name": "TimmyBishop", "user-since": datetime("2011-11-08T13:46:03.000Z"), "user-since-copy": datetime("2011-11-08T13:46:03.000Z"), "friend-ids": {{ 42987870, 44400071, 27388256, 10579275, 12546323, 23276512, 382419, 4466999, 8068553, 33814105, 14872828, 35038629, 43462816, 44037440 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2001-08-08"), "end-date": null } ] }
+{ "id": 11290987, "id-copy": 11290987, "alias": "Ilana", "name": "IlanaTedrow", "user-since": datetime("2009-03-03T00:10:34.000Z"), "user-since-copy": datetime("2009-03-03T00:10:34.000Z"), "friend-ids": {{ 20902982, 27972021, 22354642, 32382609, 18711912, 17070293 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2005-11-28"), "end-date": date("2009-09-17") } ] }
+{ "id": 11293477, "id-copy": 11293477, "alias": "Tamzen", "name": "TamzenWheeler", "user-since": datetime("2006-02-25T23:55:58.000Z"), "user-since-copy": datetime("2006-02-25T23:55:58.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-05-19"), "end-date": date("2011-03-06") } ] }
+{ "id": 11297359, "id-copy": 11297359, "alias": "Perry", "name": "PerryLowe", "user-since": datetime("2005-12-28T02:16:57.000Z"), "user-since-copy": datetime("2005-12-28T02:16:57.000Z"), "friend-ids": {{ 33439767 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2007-08-11"), "end-date": date("2009-05-16") } ] }
+{ "id": 11313361, "id-copy": 11313361, "alias": "Lashawn", "name": "LashawnSchuth", "user-since": datetime("2006-08-24T02:37:43.000Z"), "user-since-copy": datetime("2006-08-24T02:37:43.000Z"), "friend-ids": {{ 3844342, 31605302, 11335667, 3890958 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2001-04-04"), "end-date": date("2006-12-03") } ] }
+{ "id": 11330215, "id-copy": 11330215, "alias": "Tilly", "name": "TillyMckinnon", "user-since": datetime("2011-04-13T10:13:13.000Z"), "user-since-copy": datetime("2011-04-13T10:13:13.000Z"), "friend-ids": {{ 5559510, 31907101, 45791333, 35002065, 1302921, 37193818, 32812039, 41322357, 20631502 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2001-03-22"), "end-date": date("2008-08-22") } ] }
+{ "id": 11333794, "id-copy": 11333794, "alias": "Yung", "name": "YungNash", "user-since": datetime("2010-06-08T17:32:35.000Z"), "user-since-copy": datetime("2010-06-08T17:32:35.000Z"), "friend-ids": {{ 11329358, 14452899, 15459758, 31785934, 15405998, 17431717, 36883854, 1230831, 17690420, 45243495, 31580409, 15264731, 10067263, 20381783, 41240146, 2883831, 29492394, 38409147, 35853447, 26151247 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2002-06-24"), "end-date": date("2010-03-23") } ] }
+{ "id": 11348356, "id-copy": 11348356, "alias": "Chery", "name": "CherySandford", "user-since": datetime("2011-04-23T21:22:21.000Z"), "user-since-copy": datetime("2011-04-23T21:22:21.000Z"), "friend-ids": {{ 14076544, 42221517 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2000-07-25"), "end-date": null } ] }
+{ "id": 11350432, "id-copy": 11350432, "alias": "Fletcher", "name": "FletcherRowley", "user-since": datetime("2012-01-22T12:30:57.000Z"), "user-since-copy": datetime("2012-01-22T12:30:57.000Z"), "friend-ids": {{ 43655299, 46172971, 29175610, 22537183, 30612976, 21304031, 40531272, 6719806, 42232806, 18593968, 29334159 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2002-02-17"), "end-date": date("2011-03-16") } ] }
+{ "id": 11366131, "id-copy": 11366131, "alias": "Cayley", "name": "CayleyGronko", "user-since": datetime("2005-03-06T13:24:19.000Z"), "user-since-copy": datetime("2005-03-06T13:24:19.000Z"), "friend-ids": {{ 26623267, 47792710, 27975124, 19721566, 45092752, 32954140, 25835098 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2006-09-14"), "end-date": date("2010-06-02") } ] }
+{ "id": 11370337, "id-copy": 11370337, "alias": "Devin", "name": "DevinWatson", "user-since": datetime("2009-07-19T11:47:07.000Z"), "user-since-copy": datetime("2009-07-19T11:47:07.000Z"), "friend-ids": {{ 25117468, 31957773, 46217915, 26169035, 34203342, 32134285, 10572760, 10974016, 33771064, 4177645, 4910095, 18301833, 15264956, 5806057, 37899843, 35459189, 4391801, 34940818 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2008-06-19"), "end-date": null } ] }
+{ "id": 11378911, "id-copy": 11378911, "alias": "Courtney", "name": "CourtneyBashline", "user-since": datetime("2010-10-21T06:13:06.000Z"), "user-since-copy": datetime("2010-10-21T06:13:06.000Z"), "friend-ids": {{ 19627264, 13699162 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2002-06-21"), "end-date": null } ] }
+{ "id": 11400016, "id-copy": 11400016, "alias": "Beaumont", "name": "BeaumontMiller", "user-since": datetime("2008-05-12T07:13:22.000Z"), "user-since-copy": datetime("2008-05-12T07:13:22.000Z"), "friend-ids": {{ 41935126, 36767417, 10582797, 47501456, 43527117, 2821865, 27905409, 13531461, 16278289, 9565333, 15686197, 15195167, 29350985, 8804024, 31606110, 44124513, 15106563, 26509959, 47480296, 13623445, 17378703, 33568332, 19922072, 12746355 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2002-03-04"), "end-date": null } ] }
+{ "id": 11404780, "id-copy": 11404780, "alias": "Carol", "name": "CarolCox", "user-since": datetime("2009-07-07T23:58:07.000Z"), "user-since-copy": datetime("2009-07-07T23:58:07.000Z"), "friend-ids": {{ 41450896, 12332484, 18515318, 39039576, 2336271, 47313837, 4655597, 40110200, 7357446, 24291515, 8898678, 28911118, 20372890, 1296082, 42558011, 5719716, 6830197 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2005-01-14"), "end-date": null } ] }
+{ "id": 11415055, "id-copy": 11415055, "alias": "Zavia", "name": "ZaviaLombardi", "user-since": datetime("2006-01-10T02:11:24.000Z"), "user-since-copy": datetime("2006-01-10T02:11:24.000Z"), "friend-ids": {{ 25953753, 952678, 31067065 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2003-06-27"), "end-date": date("2010-07-02") } ] }
+{ "id": 11423752, "id-copy": 11423752, "alias": "Eliott", "name": "EliottRoche", "user-since": datetime("2007-07-01T04:36:16.000Z"), "user-since-copy": datetime("2007-07-01T04:36:16.000Z"), "friend-ids": {{ 34273508, 10643569, 13667612, 19808579, 46658485, 43209365, 7962014, 24567991, 25086057 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2005-11-26"), "end-date": null } ] }
+{ "id": 11426248, "id-copy": 11426248, "alias": "Chryssa", "name": "ChryssaHincken", "user-since": datetime("2005-06-16T01:11:36.000Z"), "user-since-copy": datetime("2005-06-16T01:11:36.000Z"), "friend-ids": {{ 47119545 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2003-11-20"), "end-date": date("2003-10-07") } ] }
+{ "id": 11428300, "id-copy": 11428300, "alias": "Major", "name": "MajorGreenawalt", "user-since": datetime("2006-12-02T06:43:13.000Z"), "user-since-copy": datetime("2006-12-02T06:43:13.000Z"), "friend-ids": {{ 8021918, 4810021, 34724015, 45030049, 36575685, 44527472 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2002-04-17"), "end-date": null } ] }
+{ "id": 11447332, "id-copy": 11447332, "alias": "Sherisse", "name": "SherisseMaugham", "user-since": datetime("2012-02-09T14:21:08.000Z"), "user-since-copy": datetime("2012-02-09T14:21:08.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2011-09-16"), "end-date": null } ] }
+{ "id": 11452525, "id-copy": 11452525, "alias": "Suzanna", "name": "SuzannaOlphert", "user-since": datetime("2005-10-22T04:41:20.000Z"), "user-since-copy": datetime("2005-10-22T04:41:20.000Z"), "friend-ids": {{ 44250347, 21517625, 10831891, 23365285, 2000581, 43387385, 40167252, 25288275, 6768341, 36116792, 10670805 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2001-10-21"), "end-date": date("2005-03-11") } ] }
+{ "id": 11456404, "id-copy": 11456404, "alias": "Lonny", "name": "LonnyUllman", "user-since": datetime("2008-10-19T03:05:07.000Z"), "user-since-copy": datetime("2008-10-19T03:05:07.000Z"), "friend-ids": {{ 30675414, 44654756, 8273748, 12998719, 20082930 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2009-11-02"), "end-date": date("2011-05-11") } ] }
+{ "id": 11458594, "id-copy": 11458594, "alias": "Rosaline", "name": "RosalineHawker", "user-since": datetime("2006-06-07T01:36:07.000Z"), "user-since-copy": datetime("2006-06-07T01:36:07.000Z"), "friend-ids": {{ 13674953, 43755185, 20151836, 40023637, 35564429, 45196295, 33392303, 2080473, 6786170, 42815553, 10811200, 5050190, 20987923, 32613675 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2002-06-05"), "end-date": null } ] }
+{ "id": 11468158, "id-copy": 11468158, "alias": "Pamelia", "name": "PameliaShaner", "user-since": datetime("2005-07-11T18:28:07.000Z"), "user-since-copy": datetime("2005-07-11T18:28:07.000Z"), "friend-ids": {{ 8892753, 24751024, 7162523, 38425260, 8752332, 23371746, 6673241, 22278741, 46403700 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2006-02-04"), "end-date": null } ] }
+{ "id": 11481961, "id-copy": 11481961, "alias": "Ralph", "name": "RalphMinnie", "user-since": datetime("2008-09-03T03:36:09.000Z"), "user-since-copy": datetime("2008-09-03T03:36:09.000Z"), "friend-ids": {{ 28795092, 15427393, 13323116, 6103928, 22507606, 38931008, 8419762, 30922606, 11217439, 41769747, 19668638, 26796252, 26750627, 4855539, 11170229, 30124829, 16596482, 15728547, 46139530, 43784722, 20640234, 22313927, 16136087, 39688415 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2006-02-13"), "end-date": null } ] }
+{ "id": 11529364, "id-copy": 11529364, "alias": "Rufus", "name": "RufusGreen", "user-since": datetime("2009-04-14T15:51:24.000Z"), "user-since-copy": datetime("2009-04-14T15:51:24.000Z"), "friend-ids": {{ 5011595 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2000-09-25"), "end-date": date("2004-08-22") } ] }
+{ "id": 11529952, "id-copy": 11529952, "alias": "Charles", "name": "CharlesHarrow", "user-since": datetime("2008-11-24T19:27:12.000Z"), "user-since-copy": datetime("2008-11-24T19:27:12.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2005-06-08"), "end-date": date("2011-10-27") } ] }
+{ "id": 11536582, "id-copy": 11536582, "alias": "Deon", "name": "DeonBickerson", "user-since": datetime("2007-05-18T18:12:00.000Z"), "user-since-copy": datetime("2007-05-18T18:12:00.000Z"), "friend-ids": {{ 2848304, 6359671, 29695732, 42414044, 3277185, 17642866, 47064497, 32240400, 43486181, 5049864, 22831246, 9259974, 17502793, 29955647, 6928887, 19609966 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2009-01-20"), "end-date": date("2009-03-12") } ] }
+{ "id": 11570617, "id-copy": 11570617, "alias": "Deshawn", "name": "DeshawnBashline", "user-since": datetime("2006-04-14T01:05:38.000Z"), "user-since-copy": datetime("2006-04-14T01:05:38.000Z"), "friend-ids": {{ 9319940, 45556479, 44222390, 22928539, 27909778, 21162548, 8657905, 15375082, 38338906, 21416203, 7519884, 30405265, 32148274, 35560776, 29973785, 19277384, 44256954, 40425041, 30328494, 39977803, 40280359, 3079013, 18841024, 23001903 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2009-02-14"), "end-date": null } ] }
+{ "id": 11573350, "id-copy": 11573350, "alias": "Sommer", "name": "SommerGregory", "user-since": datetime("2007-08-25T21:50:51.000Z"), "user-since-copy": datetime("2007-08-25T21:50:51.000Z"), "friend-ids": {{ 6622046, 40071999, 24631984, 42427860, 13378139, 27659078, 32813734, 20145238, 15342806, 9562288, 24211264, 29951003, 3620479, 43701781, 22474191, 6298296, 4047189, 27133942, 8058121, 9928231, 31835361, 6234235, 6100660, 1575061 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-12-09"), "end-date": date("2010-01-16") } ] }
+{ "id": 11595592, "id-copy": 11595592, "alias": "Bert", "name": "BertAtkinson", "user-since": datetime("2011-09-03T07:24:42.000Z"), "user-since-copy": datetime("2011-09-03T07:24:42.000Z"), "friend-ids": {{ 36724561, 45824456, 33567747, 21400268, 11419574, 47463040, 6480088, 45216774, 26857982, 7140352, 1884512, 29610211, 2626672, 41371388, 43582371, 42445087, 14734124, 3580372, 40134022 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2006-06-27"), "end-date": date("2007-06-07") } ] }
+{ "id": 11616628, "id-copy": 11616628, "alias": "Jessamine", "name": "JessamineWolff", "user-since": datetime("2008-05-03T17:05:35.000Z"), "user-since-copy": datetime("2008-05-03T17:05:35.000Z"), "friend-ids": {{ 38285911, 42183685, 11422759, 25927239, 22771435, 47814309, 43146385, 39761181, 1670925, 15764683, 8068597, 3561105 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2007-07-26"), "end-date": date("2010-03-16") } ] }
+{ "id": 11625859, "id-copy": 11625859, "alias": "Zacharias", "name": "ZachariasSanner", "user-since": datetime("2007-06-12T21:21:21.000Z"), "user-since-copy": datetime("2007-06-12T21:21:21.000Z"), "friend-ids": {{ 13379571, 45822651, 39352555, 11549959, 24329960, 2142134, 15486962, 43011509, 46074449, 9322703 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2007-05-15"), "end-date": null } ] }
+{ "id": 11638618, "id-copy": 11638618, "alias": "Garfield", "name": "GarfieldHardie", "user-since": datetime("2007-07-05T04:44:27.000Z"), "user-since-copy": datetime("2007-07-05T04:44:27.000Z"), "friend-ids": {{ 47307628, 3109848, 30936899, 7173119, 33551634, 24239136, 11619168, 633835, 34791947, 12052833, 19798108, 3426648, 395456, 18555868, 18509839, 8340275, 14943912, 42330581, 313099, 25632353, 27912788, 20281899, 8961605, 13625222 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2001-02-24"), "end-date": null } ] }
+{ "id": 11646016, "id-copy": 11646016, "alias": "Millard", "name": "MillardCribbs", "user-since": datetime("2012-07-01T13:28:56.000Z"), "user-since-copy": datetime("2012-07-01T13:28:56.000Z"), "friend-ids": {{ 29358027, 24800104, 1146956, 29116484, 12223225, 6324161, 46576675 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2004-04-28"), "end-date": null } ] }
+{ "id": 11666128, "id-copy": 11666128, "alias": "Mathilda", "name": "MathildaBurris", "user-since": datetime("2006-01-04T14:30:09.000Z"), "user-since-copy": datetime("2006-01-04T14:30:09.000Z"), "friend-ids": {{ 21229678, 40152290, 2867638, 27694777, 34054129, 47727334, 39805693, 9084777, 37744206, 47011794, 2190990, 19109454 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2007-09-14"), "end-date": date("2007-03-17") } ] }
+{ "id": 11672578, "id-copy": 11672578, "alias": "Juli", "name": "JuliMcclymonds", "user-since": datetime("2010-07-17T13:53:57.000Z"), "user-since-copy": datetime("2010-07-17T13:53:57.000Z"), "friend-ids": {{ 16548983, 7350585, 44497037 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2003-05-23"), "end-date": date("2009-08-01") } ] }
+{ "id": 11674741, "id-copy": 11674741, "alias": "Soon", "name": "SoonBillimek", "user-since": datetime("2009-03-02T12:08:16.000Z"), "user-since-copy": datetime("2009-03-02T12:08:16.000Z"), "friend-ids": {{ 26069920, 16634341, 13963293, 27425934, 19271848, 22444876, 42264629, 39307655, 21118192, 27961060, 12398172, 13202296, 23221559, 34323488, 1588557, 42672479, 19548482, 28266272, 6241122, 13633490 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2006-01-19"), "end-date": date("2011-03-25") } ] }
+{ "id": 11676574, "id-copy": 11676574, "alias": "Isidore", "name": "IsidoreCatlay", "user-since": datetime("2012-08-26T08:28:08.000Z"), "user-since-copy": datetime("2012-08-26T08:28:08.000Z"), "friend-ids": {{ 46189001 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2001-03-07"), "end-date": null } ] }
+{ "id": 11694928, "id-copy": 11694928, "alias": "Anne", "name": "AnnePritchard", "user-since": datetime("2005-05-25T23:02:45.000Z"), "user-since-copy": datetime("2005-05-25T23:02:45.000Z"), "friend-ids": {{ 4000537, 32410978, 2682612, 1214946, 38250943, 36272447, 14182545, 27782322, 2714608, 38315875 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2011-02-22"), "end-date": date("2011-11-07") } ] }
+{ "id": 11735830, "id-copy": 11735830, "alias": "Maryvonne", "name": "MaryvonneHarrold", "user-since": datetime("2007-12-03T06:30:43.000Z"), "user-since-copy": datetime("2007-12-03T06:30:43.000Z"), "friend-ids": {{ 27842540, 46624942, 21701969, 33750891, 28523702, 38840881, 1497785, 32357938, 19740312, 1880841, 41116687, 35621654, 46917268, 14610853, 33099367, 8710534 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2004-10-04"), "end-date": null } ] }
+{ "id": 11762239, "id-copy": 11762239, "alias": "Guillermo", "name": "GuillermoCallison", "user-since": datetime("2009-02-12T13:46:40.000Z"), "user-since-copy": datetime("2009-02-12T13:46:40.000Z"), "friend-ids": {{ 3494924, 650832, 22099424, 11629223, 45581083, 206762, 27794516, 7639789, 31794781, 22985617, 17273963, 9120417, 9496942, 47474589, 47872578, 34639130, 37695869, 41346670, 7789418, 24870369, 31562430, 2414862, 41928569 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2008-07-26"), "end-date": null } ] }
+{ "id": 11763463, "id-copy": 11763463, "alias": "Haven", "name": "HavenRaub", "user-since": datetime("2012-03-01T12:41:53.000Z"), "user-since-copy": datetime("2012-03-01T12:41:53.000Z"), "friend-ids": {{ 19981286 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2001-04-26"), "end-date": null } ] }
+{ "id": 11830822, "id-copy": 11830822, "alias": "Lincoln", "name": "LincolnFuchs", "user-since": datetime("2008-01-22T19:08:51.000Z"), "user-since-copy": datetime("2008-01-22T19:08:51.000Z"), "friend-ids": {{ 29014579, 29789039, 2225447, 37872940, 37026231, 3223799, 40601178 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2006-01-14"), "end-date": date("2010-04-24") } ] }
+{ "id": 11839117, "id-copy": 11839117, "alias": "Kyra", "name": "KyraMcdonald", "user-since": datetime("2010-07-08T20:46:49.000Z"), "user-since-copy": datetime("2010-07-08T20:46:49.000Z"), "friend-ids": {{ 42933043, 41665211, 13075886, 36147059, 20127919, 31449381, 47427643, 24399833, 16541120, 38909218, 15609877, 46802599, 31772232, 46743670 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2005-06-08"), "end-date": date("2007-11-11") } ] }
+{ "id": 11862502, "id-copy": 11862502, "alias": "Innocent", "name": "InnocentWilliamson", "user-since": datetime("2005-06-09T18:44:51.000Z"), "user-since-copy": datetime("2005-06-09T18:44:51.000Z"), "friend-ids": {{ 14750408, 36287814, 21197416, 34246775, 18776860, 32777856, 46956112, 18578056, 13053407, 3282278, 29812571, 25299530, 47168979, 6027296, 10540009 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2010-05-20"), "end-date": date("2010-01-24") } ] }
+{ "id": 11912419, "id-copy": 11912419, "alias": "Wallis", "name": "WallisFuchs", "user-since": datetime("2012-01-07T08:13:18.000Z"), "user-since-copy": datetime("2012-01-07T08:13:18.000Z"), "friend-ids": {{ 11115387, 19639311, 33957302, 8746808, 20140328, 35866755, 29492622, 24246926, 14412186, 1610423, 1139443, 23667812, 6972455, 18354247, 7072427, 43742595, 20711654, 7179925, 66544, 12066267, 8914321, 35602734 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2008-10-23"), "end-date": date("2008-06-18") } ] }
+{ "id": 11914129, "id-copy": 11914129, "alias": "Ebenezer", "name": "EbenezerMonahan", "user-since": datetime("2006-01-08T08:17:51.000Z"), "user-since-copy": datetime("2006-01-08T08:17:51.000Z"), "friend-ids": {{ 9692770 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2002-10-22"), "end-date": date("2005-07-17") } ] }
+{ "id": 11921524, "id-copy": 11921524, "alias": "Mickey", "name": "MickeySybilla", "user-since": datetime("2012-03-28T17:05:25.000Z"), "user-since-copy": datetime("2012-03-28T17:05:25.000Z"), "friend-ids": {{ 40813978, 14172552, 40702786, 929262, 2220334, 33077762, 20716547, 11400385, 21916926, 38422356, 13378381, 32362984, 8162369, 8965084, 37823302, 3542211, 29294304, 37672739, 28359647 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2007-09-27"), "end-date": null } ] }
+{ "id": 11937787, "id-copy": 11937787, "alias": "Addison", "name": "AddisonEckert", "user-since": datetime("2007-04-26T01:06:38.000Z"), "user-since-copy": datetime("2007-04-26T01:06:38.000Z"), "friend-ids": {{ 6446414, 23134374, 38952228, 25368200, 47868440, 29231397, 15672064, 2482344, 22824732, 13563448, 43826877 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2009-10-09"), "end-date": null } ] }
+{ "id": 11965318, "id-copy": 11965318, "alias": "Donella", "name": "DonellaPriebe", "user-since": datetime("2010-10-25T19:45:41.000Z"), "user-since-copy": datetime("2010-10-25T19:45:41.000Z"), "friend-ids": {{ 40521325 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2010-11-19"), "end-date": date("2011-08-18") } ] }
+{ "id": 11989228, "id-copy": 11989228, "alias": "Jaden", "name": "JadenKelley", "user-since": datetime("2006-11-12T15:45:55.000Z"), "user-since-copy": datetime("2006-11-12T15:45:55.000Z"), "friend-ids": {{ 39881086, 47143027, 9394301, 17338199, 16961896, 6602092, 46708527, 24050942, 20543677, 13309656 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2003-10-19"), "end-date": null } ] }
+{ "id": 11989660, "id-copy": 11989660, "alias": "Rolland", "name": "RollandGarneis", "user-since": datetime("2008-09-16T19:54:32.000Z"), "user-since-copy": datetime("2008-09-16T19:54:32.000Z"), "friend-ids": {{ 30959592, 6160903, 27316367, 6518756, 23008668, 36942525, 39489068, 8710310, 17726852, 72593, 15440937, 4901728, 28916846, 38257093, 28414859, 8857050 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2008-02-11"), "end-date": null } ] }
+{ "id": 9001816, "id-copy": 9001816, "alias": "Concordia", "name": "ConcordiaThomlinson", "user-since": datetime("2006-04-13T03:30:17.000Z"), "user-since-copy": datetime("2006-04-13T03:30:17.000Z"), "friend-ids": {{ 31001079, 10620343, 29160614, 8991085, 45471665, 865015, 11592391, 33106281, 15448665, 29325047, 47814022, 4562661, 11895808, 41974900 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2002-03-25"), "end-date": null } ] }
+{ "id": 9005248, "id-copy": 9005248, "alias": "Jervis", "name": "JervisWarrick", "user-since": datetime("2007-02-06T17:54:17.000Z"), "user-since-copy": datetime("2007-02-06T17:54:17.000Z"), "friend-ids": {{ 5038062, 15101135, 28136073, 10706469, 8706391, 10623870, 1759405, 37020186, 17173998, 14985805, 19308437, 43696985, 46650868, 25621415, 14252531, 44491166, 42536769, 33614525, 34665072, 640793 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2010-08-21"), "end-date": null } ] }
+{ "id": 9012382, "id-copy": 9012382, "alias": "Laureen", "name": "LaureenOneal", "user-since": datetime("2009-12-10T22:17:58.000Z"), "user-since-copy": datetime("2009-12-10T22:17:58.000Z"), "friend-ids": {{ 25012654, 4572832, 38401260, 3015853, 42975956, 16328675, 39626774, 26936410, 15112607, 3302431 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2007-11-10"), "end-date": null } ] }
+{ "id": 9012778, "id-copy": 9012778, "alias": "Godfrey", "name": "GodfreyBraun", "user-since": datetime("2010-03-18T19:15:53.000Z"), "user-since-copy": datetime("2010-03-18T19:15:53.000Z"), "friend-ids": {{ 3867712, 22078166 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2005-10-02"), "end-date": null } ] }
+{ "id": 9035089, "id-copy": 9035089, "alias": "Marylyn", "name": "MarylynSteele", "user-since": datetime("2005-04-24T04:55:25.000Z"), "user-since-copy": datetime("2005-04-24T04:55:25.000Z"), "friend-ids": {{ 4250473, 16568038, 10872744, 32513859, 37267973, 2225211, 45148996, 1080441, 13013464, 10394988, 3316854, 8183563, 228753, 6849521, 8786964, 21073526 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2011-02-11"), "end-date": date("2011-10-08") } ] }
+{ "id": 9039973, "id-copy": 9039973, "alias": "Desmond", "name": "DesmondRice", "user-since": datetime("2008-04-17T12:00:38.000Z"), "user-since-copy": datetime("2008-04-17T12:00:38.000Z"), "friend-ids": {{ 16128090, 28937536, 30905098, 25666304, 23272582, 29438991, 42040849, 42396891, 9345677, 9260055, 17415621, 31581557, 1249365, 20734436, 2341357, 36307325, 20347771, 23723655 }}, "employment": [ { "organization-name": "Zimcone", "start-date": date("2002-10-24"), "end-date": date("2008-02-24") } ] }
+{ "id": 9043201, "id-copy": 9043201, "alias": "Eliseo", "name": "EliseoBagley", "user-since": datetime("2007-05-17T10:44:18.000Z"), "user-since-copy": datetime("2007-05-17T10:44:18.000Z"), "friend-ids": {{ 41250222, 28415639, 40825493, 11902499, 39161617, 16612650, 39102228, 46013732, 42664763, 20165539, 40891614, 2887877, 27999503, 5059039, 9617378, 16378780, 21987749 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2003-05-26"), "end-date": null } ] }
+{ "id": 9081124, "id-copy": 9081124, "alias": "Aureole", "name": "AureoleChappel", "user-since": datetime("2005-03-24T18:14:35.000Z"), "user-since-copy": datetime("2005-03-24T18:14:35.000Z"), "friend-ids": {{ 16199402, 2970920 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2005-07-19"), "end-date": date("2011-04-02") } ] }
+{ "id": 9099376, "id-copy": 9099376, "alias": "Tena", "name": "TenaKline", "user-since": datetime("2011-10-20T14:46:29.000Z"), "user-since-copy": datetime("2011-10-20T14:46:29.000Z"), "friend-ids": {{ 28615752, 16589994, 24896126, 32768352, 40921310, 22643822, 39206554, 45652466, 17237997, 44705249, 30599864, 17750741, 14758376, 4842744 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2000-03-18"), "end-date": null } ] }
+{ "id": 9107137, "id-copy": 9107137, "alias": "Woodrow", "name": "WoodrowMueller", "user-since": datetime("2012-06-15T04:53:52.000Z"), "user-since-copy": datetime("2012-06-15T04:53:52.000Z"), "friend-ids": {{ 39459662, 1343459, 16606290, 21443457, 29053037, 28244658, 27954195, 9411908, 2059678, 24579828, 40955404 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2002-02-11"), "end-date": null } ] }
+{ "id": 9112336, "id-copy": 9112336, "alias": "Marlin", "name": "MarlinRosenstiehl", "user-since": datetime("2010-09-26T04:27:50.000Z"), "user-since-copy": datetime("2010-09-26T04:27:50.000Z"), "friend-ids": {{ 10225686, 16259250, 11552542, 28661586, 8900635, 27988260 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2011-12-05"), "end-date": null } ] }
+{ "id": 9146107, "id-copy": 9146107, "alias": "Femie", "name": "FemieBurns", "user-since": datetime("2007-05-05T03:23:12.000Z"), "user-since-copy": datetime("2007-05-05T03:23:12.000Z"), "friend-ids": {{ 38688633, 2489245, 43502175, 34373436, 11854240, 23544813, 44263720, 20953878, 37021620, 16028559, 20673451, 46975172, 47409532, 44524395 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2008-07-26"), "end-date": null } ] }
+{ "id": 9160906, "id-copy": 9160906, "alias": "Cathryn", "name": "CathrynReamer", "user-since": datetime("2010-10-08T06:24:51.000Z"), "user-since-copy": datetime("2010-10-08T06:24:51.000Z"), "friend-ids": {{ 30962953 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2006-10-28"), "end-date": date("2010-03-14") } ] }
+{ "id": 9170767, "id-copy": 9170767, "alias": "Noble", "name": "NobleByers", "user-since": datetime("2012-04-19T03:21:33.000Z"), "user-since-copy": datetime("2012-04-19T03:21:33.000Z"), "friend-ids": {{ 17464807, 11911237, 31984348, 14323306, 21828766, 24212960, 3269277, 24648466, 30032203, 15837021, 12033801, 3899014, 6105665, 4416812, 33902540, 9640452, 3739829, 14414940, 36838129, 7327467, 35420130, 24031049 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2007-08-11"), "end-date": null } ] }
+{ "id": 9179413, "id-copy": 9179413, "alias": "Benton", "name": "BentonMorland", "user-since": datetime("2006-02-08T13:43:03.000Z"), "user-since-copy": datetime("2006-02-08T13:43:03.000Z"), "friend-ids": {{ 25229017, 22411534, 46862190, 17238544, 10875646, 19572187, 9889710, 23940269, 24489112, 7997331, 8866147, 29705622, 35336434, 14756488, 40059408, 32606759, 37546068, 24168033, 20761302, 45465986, 27519909, 23920570, 3984052, 38799668 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-12-05"), "end-date": null } ] }
+{ "id": 9187549, "id-copy": 9187549, "alias": "Lenny", "name": "LennyField", "user-since": datetime("2008-09-11T10:50:10.000Z"), "user-since-copy": datetime("2008-09-11T10:50:10.000Z"), "friend-ids": {{ 26505249, 4392946, 32062169, 45628101, 22865593, 4982483, 13425537, 18846467, 36122039, 2998293, 19787439, 22246499, 43133873, 30573462, 36272473, 41691126, 43929640, 43759980, 25546305 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2004-08-13"), "end-date": date("2010-03-08") } ] }
+{ "id": 9205834, "id-copy": 9205834, "alias": "Tristin", "name": "TristinWalker", "user-since": datetime("2012-04-25T01:08:05.000Z"), "user-since-copy": datetime("2012-04-25T01:08:05.000Z"), "friend-ids": {{ 2222398, 15073251, 16222879, 24405969, 32651599, 44500557, 31699173, 41724026, 1745441, 9674348, 29594086, 26580583, 42258300, 36027050, 3204087, 2147469, 36519580 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2007-02-09"), "end-date": null } ] }
+{ "id": 9207832, "id-copy": 9207832, "alias": "Tammy", "name": "TammyHozier", "user-since": datetime("2005-08-24T14:34:19.000Z"), "user-since-copy": datetime("2005-08-24T14:34:19.000Z"), "friend-ids": {{ 26919119, 35729176, 28949827 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2006-09-14"), "end-date": null } ] }
+{ "id": 9209866, "id-copy": 9209866, "alias": "Timothy", "name": "TimothyBuck", "user-since": datetime("2009-11-07T14:19:12.000Z"), "user-since-copy": datetime("2009-11-07T14:19:12.000Z"), "friend-ids": {{ 43082021, 25019103, 26061770, 7134151, 17663441, 35230064, 731481, 6719229, 23303796, 40777269 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2000-04-03"), "end-date": date("2000-04-20") } ] }
+{ "id": 9210847, "id-copy": 9210847, "alias": "Kristeen", "name": "KristeenShaffer", "user-since": datetime("2008-01-04T12:31:50.000Z"), "user-since-copy": datetime("2008-01-04T12:31:50.000Z"), "friend-ids": {{ 662954, 18313322, 10737685, 5498351, 24795605, 4497605, 45729062, 31007969, 16211490, 19408104, 5882137, 12084923, 14143383, 31263672, 32404691, 8973685, 32756191, 3822704 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2009-12-07"), "end-date": date("2010-02-08") } ] }
+{ "id": 9226960, "id-copy": 9226960, "alias": "Irish", "name": "IrishJohnson", "user-since": datetime("2009-09-07T21:02:01.000Z"), "user-since-copy": datetime("2009-09-07T21:02:01.000Z"), "friend-ids": {{ 4920892, 15681759, 19110917, 26620361, 34712468, 40890326, 20312413 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2009-11-11"), "end-date": null } ] }
+{ "id": 9239515, "id-copy": 9239515, "alias": "Precious", "name": "PreciousWeingarten", "user-since": datetime("2006-07-03T10:28:56.000Z"), "user-since-copy": datetime("2006-07-03T10:28:56.000Z"), "friend-ids": {{ 20459132, 9181399, 30604442, 45266959, 31805782, 8190732, 46444663, 46572075, 43980715, 42547186, 21087158, 38075989, 32228414, 25466991, 4929897, 33467622, 35742242, 7150399, 16997658, 18543557, 11799062 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2003-02-15"), "end-date": null } ] }
+{ "id": 9318094, "id-copy": 9318094, "alias": "Carlo", "name": "CarloKelley", "user-since": datetime("2012-07-19T09:18:41.000Z"), "user-since-copy": datetime("2012-07-19T09:18:41.000Z"), "friend-ids": {{ 39873731, 29304807, 519851, 16423529, 10838418, 9915172, 3040071, 39730361, 23320290, 20572900, 7293676, 35037765, 1744053, 38875858 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2008-08-15"), "end-date": null } ] }
+{ "id": 9329272, "id-copy": 9329272, "alias": "Nonie", "name": "NonieStafford", "user-since": datetime("2005-10-01T21:12:24.000Z"), "user-since-copy": datetime("2005-10-01T21:12:24.000Z"), "friend-ids": {{ 42745071, 14744035, 37742648, 31179205, 28520118, 32828516, 2726599, 1667680 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2004-06-21"), "end-date": null } ] }
+{ "id": 9361930, "id-copy": 9361930, "alias": "Leonard", "name": "LeonardAshbaugh", "user-since": datetime("2008-06-13T07:49:33.000Z"), "user-since-copy": datetime("2008-06-13T07:49:33.000Z"), "friend-ids": {{ 33929562, 22722370, 18562061, 44346144, 38834006, 1660309, 17690686, 8299074, 13219630, 42802095, 2203402, 47180979, 43715995, 24339545, 42132653, 32010945, 18200992, 5115504 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2012-04-15"), "end-date": null } ] }
+{ "id": 9366253, "id-copy": 9366253, "alias": "Emma", "name": "EmmaKnisely", "user-since": datetime("2012-07-08T20:39:00.000Z"), "user-since-copy": datetime("2012-07-08T20:39:00.000Z"), "friend-ids": {{ 40874500, 35049897, 29559982, 42737582, 11405173, 38919458, 26268603, 38582942, 13758558, 16949073 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2009-11-18"), "end-date": null } ] }
+{ "id": 9417499, "id-copy": 9417499, "alias": "Wendell", "name": "WendellJoyce", "user-since": datetime("2011-07-25T14:30:30.000Z"), "user-since-copy": datetime("2011-07-25T14:30:30.000Z"), "friend-ids": {{ 10079972, 29246113, 40533159, 31279768, 31969044, 46120195, 35004468, 24465042, 2702879, 44166678, 20176481, 32056309, 38254930, 20950061, 4687108 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2006-03-18"), "end-date": null } ] }
+{ "id": 9418882, "id-copy": 9418882, "alias": "Laurine", "name": "LaurineCowart", "user-since": datetime("2012-06-14T22:26:09.000Z"), "user-since-copy": datetime("2012-06-14T22:26:09.000Z"), "friend-ids": {{ 19430214, 17084414, 12678029, 1783933, 42580022, 26274674, 13661281, 31117329, 19971039, 43840305, 42672247, 17088417, 31128028, 41009670, 16020772 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2002-03-20"), "end-date": null } ] }
+{ "id": 9426544, "id-copy": 9426544, "alias": "Joshawa", "name": "JoshawaHiles", "user-since": datetime("2012-04-28T09:48:20.000Z"), "user-since-copy": datetime("2012-04-28T09:48:20.000Z"), "friend-ids": {{ 16780903 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2002-07-01"), "end-date": null } ] }
+{ "id": 9440818, "id-copy": 9440818, "alias": "Poppy", "name": "PoppyBoyer", "user-since": datetime("2007-06-09T08:15:05.000Z"), "user-since-copy": datetime("2007-06-09T08:15:05.000Z"), "friend-ids": {{ 10721272, 26882431, 45774996, 44725231, 34694934, 28877797, 12922671, 16078039, 43902220, 27311426, 34146150, 39285332, 7343219, 17482231, 15496713, 12439079, 18097780, 30046636, 16951144, 27968612 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2006-11-17"), "end-date": null } ] }
+{ "id": 9450532, "id-copy": 9450532, "alias": "Troy", "name": "TroyKoepple", "user-since": datetime("2011-05-10T09:56:46.000Z"), "user-since-copy": datetime("2011-05-10T09:56:46.000Z"), "friend-ids": {{ 42029412, 18025243, 715282, 501115, 38550360, 39016114, 31451417, 38836992, 13665836, 17286159, 28850827, 17241066, 41893804, 39172781, 4523003, 28542863, 25386847, 44039032, 19593806, 607220, 26442265, 47847281 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2002-12-26"), "end-date": date("2004-04-05") } ] }
+{ "id": 9477040, "id-copy": 9477040, "alias": "Chery", "name": "CheryWatson", "user-since": datetime("2012-05-02T14:27:00.000Z"), "user-since-copy": datetime("2012-05-02T14:27:00.000Z"), "friend-ids": {{ 36360097, 36835617, 25761112, 30806900, 22340413, 16802957 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2008-12-26"), "end-date": date("2009-03-17") } ] }
+{ "id": 9482569, "id-copy": 9482569, "alias": "Marty", "name": "MartyBurnett", "user-since": datetime("2006-03-21T10:10:40.000Z"), "user-since-copy": datetime("2006-03-21T10:10:40.000Z"), "friend-ids": {{ 5791578, 3884688, 7686005 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2009-02-01"), "end-date": null } ] }
+{ "id": 9483769, "id-copy": 9483769, "alias": "Marketta", "name": "MarkettaSchere", "user-since": datetime("2006-04-02T05:48:16.000Z"), "user-since-copy": datetime("2006-04-02T05:48:16.000Z"), "friend-ids": {{ 15151816, 38432593, 14501842, 21508230, 20201815, 35434395, 46212890, 9387767, 35469959, 6671088, 38888798, 10719563, 36944652, 36703732, 9646545, 29287523, 24156038, 24502755 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2004-07-20"), "end-date": date("2006-03-10") } ] }
+{ "id": 9545461, "id-copy": 9545461, "alias": "Sandra", "name": "SandraFea", "user-since": datetime("2005-12-09T14:40:28.000Z"), "user-since-copy": datetime("2005-12-09T14:40:28.000Z"), "friend-ids": {{ 28976045 }}, "employment": [ { "organization-name": "Voltbam", "start-date": date("2012-02-02"), "end-date": null } ] }
+{ "id": 9549610, "id-copy": 9549610, "alias": "Blossom", "name": "BlossomGreif", "user-since": datetime("2010-05-03T21:08:56.000Z"), "user-since-copy": datetime("2010-05-03T21:08:56.000Z"), "friend-ids": {{ 47791115, 42952282 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2011-12-25"), "end-date": date("2011-11-27") } ] }
+{ "id": 9552016, "id-copy": 9552016, "alias": "Shantelle", "name": "ShantelleDealtry", "user-since": datetime("2006-05-03T06:49:13.000Z"), "user-since-copy": datetime("2006-05-03T06:49:13.000Z"), "friend-ids": {{ 35758396, 16562240, 23596680, 16342769, 19892813, 46485447, 25711418, 23765073, 11303996, 36451291, 17586370, 38010455, 29457199, 25847013, 12604123, 46533018, 26999208, 24740610, 35225441, 33613663 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2003-08-07"), "end-date": date("2003-07-17") } ] }
+{ "id": 9560251, "id-copy": 9560251, "alias": "Nivek", "name": "NivekJowers", "user-since": datetime("2007-02-04T08:02:07.000Z"), "user-since-copy": datetime("2007-02-04T08:02:07.000Z"), "friend-ids": {{ 15730417, 36745553, 26133088, 38675683, 14617495, 39244216, 4651791, 639869, 8377526, 15158817, 13368295, 15386494, 5649384, 8449938, 34497809, 6644713, 45481442, 27678941, 14214532, 5753112, 9991855, 25975202, 9530884, 19069924 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2003-08-15"), "end-date": null } ] }
+{ "id": 9574261, "id-copy": 9574261, "alias": "Kalysta", "name": "KalystaBeedell", "user-since": datetime("2010-01-27T14:57:31.000Z"), "user-since-copy": datetime("2010-01-27T14:57:31.000Z"), "friend-ids": {{ 5811189, 22155580, 41736564, 27399656, 40013573, 28340467, 45690668, 16097604, 9655169, 44870593 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2009-12-16"), "end-date": date("2010-10-22") } ] }
+{ "id": 9588427, "id-copy": 9588427, "alias": "Tiffany", "name": "TiffanyGeyer", "user-since": datetime("2007-09-10T11:20:53.000Z"), "user-since-copy": datetime("2007-09-10T11:20:53.000Z"), "friend-ids": {{ 31357437, 16305152, 39281885, 25249419, 434661, 13634747, 39812462, 25218908, 22362649, 41696008, 4523776, 40340358, 45330588, 299997, 11538141, 20972409, 25152923, 8627592, 33381524, 6226232 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2005-02-20"), "end-date": null } ] }
+{ "id": 9591646, "id-copy": 9591646, "alias": "Hoyt", "name": "HoytGilman", "user-since": datetime("2011-05-13T07:22:20.000Z"), "user-since-copy": datetime("2011-05-13T07:22:20.000Z"), "friend-ids": {{ 11207445 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2004-04-27"), "end-date": null } ] }
+{ "id": 9594523, "id-copy": 9594523, "alias": "Tam", "name": "TamWillcox", "user-since": datetime("2011-12-23T11:41:58.000Z"), "user-since-copy": datetime("2011-12-23T11:41:58.000Z"), "friend-ids": {{ 27383896, 20745988, 10063024, 8241427, 40299998, 32408463, 25171835, 22380586, 15344194, 25951348, 28733234, 45421004, 2273747, 2229862, 6241144, 6704115, 8659430, 47431991, 47929530, 24393021 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2001-07-27"), "end-date": null } ] }
+{ "id": 9595279, "id-copy": 9595279, "alias": "Emmaline", "name": "EmmalineSchuth", "user-since": datetime("2008-09-12T22:25:17.000Z"), "user-since-copy": datetime("2008-09-12T22:25:17.000Z"), "friend-ids": {{ 26784778, 6200196, 37440596, 12250319, 21921557, 19278082, 583040, 12012653, 21578028, 16395818, 29088493, 29578064, 37745574, 41998781, 22594273, 38002130, 2166585, 7823908, 18253304, 6162341, 40270219, 41832701, 36455204 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2009-02-13"), "end-date": null } ] }
+{ "id": 9597526, "id-copy": 9597526, "alias": "Emory", "name": "EmoryThorley", "user-since": datetime("2006-01-19T22:44:03.000Z"), "user-since-copy": datetime("2006-01-19T22:44:03.000Z"), "friend-ids": {{ 420066, 8047878, 20510786, 1639671, 22923859, 27319995, 3624690, 18526424, 45857863, 2830065, 4588990, 25531572, 17878497, 47796172, 41309806, 34307425, 10084701, 1659934, 38218970, 44720636, 43501970, 610796, 35455526, 2080900 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2011-06-18"), "end-date": date("2011-09-10") } ] }
+{ "id": 9598486, "id-copy": 9598486, "alias": "Grover", "name": "GroverNewbern", "user-since": datetime("2012-01-06T20:50:38.000Z"), "user-since-copy": datetime("2012-01-06T20:50:38.000Z"), "friend-ids": {{ 8389292, 25521744, 23387036, 38008541, 43673600, 23656679, 1401712, 39164079, 1810015, 20625744, 15651316, 23441546, 24572830, 19077921 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2009-07-28"), "end-date": date("2010-06-09") } ] }
+{ "id": 9599647, "id-copy": 9599647, "alias": "Alexandria", "name": "AlexandriaWade", "user-since": datetime("2012-06-25T06:48:48.000Z"), "user-since-copy": datetime("2012-06-25T06:48:48.000Z"), "friend-ids": {{ 20910866, 20843338, 8182424, 21070448, 43548111, 39370893, 26760127, 11135506 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2011-06-02"), "end-date": null } ] }
+{ "id": 9606691, "id-copy": 9606691, "alias": "Reva", "name": "RevaChristman", "user-since": datetime("2010-03-04T11:53:00.000Z"), "user-since-copy": datetime("2010-03-04T11:53:00.000Z"), "friend-ids": {{ 21390421 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2010-12-13"), "end-date": null } ] }
+{ "id": 9638248, "id-copy": 9638248, "alias": "Azucena", "name": "AzucenaEmrick", "user-since": datetime("2005-12-04T00:15:40.000Z"), "user-since-copy": datetime("2005-12-04T00:15:40.000Z"), "friend-ids": {{ 37210744, 43097413, 2901403, 24492031, 7887583, 42518446, 28555003, 20402754, 5506767, 22982986, 21168589, 37638670, 30930177, 43662522, 45627167, 13450586, 36757137, 46663990 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2002-07-07"), "end-date": date("2006-06-11") } ] }
+{ "id": 9646474, "id-copy": 9646474, "alias": "Lilac", "name": "LilacWoodworth", "user-since": datetime("2009-12-17T02:42:51.000Z"), "user-since-copy": datetime("2009-12-17T02:42:51.000Z"), "friend-ids": {{ 47784123, 45348808, 36392712, 9381262, 10215254, 1461251, 23038092, 44549001, 39097217, 41152823, 31758517, 19401493, 39964393, 46307214, 41683224, 39011968, 5014398, 482179, 3789628, 46257340, 36041029, 10903757, 5980810, 31935548 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2004-10-25"), "end-date": date("2005-05-04") } ] }
+{ "id": 9674677, "id-copy": 9674677, "alias": "Skye", "name": "SkyeTomlinson", "user-since": datetime("2006-02-02T19:15:10.000Z"), "user-since-copy": datetime("2006-02-02T19:15:10.000Z"), "friend-ids": {{ 24282798, 5600117, 33292938, 19518197, 11735189, 22867735, 8029689, 11269147, 7443311, 45905216, 12859442, 26944030 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2011-05-07"), "end-date": date("2011-04-19") } ] }
+{ "id": 9690049, "id-copy": 9690049, "alias": "Ahmed", "name": "AhmedVinsant", "user-since": datetime("2009-12-24T23:10:10.000Z"), "user-since-copy": datetime("2009-12-24T23:10:10.000Z"), "friend-ids": {{ 9425379, 24773026, 47645199, 12718095, 32145472, 30931581, 11512330, 46898742, 26190870, 38985851, 40692118, 34327720, 47432207 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2002-05-26"), "end-date": null } ] }
+{ "id": 9693988, "id-copy": 9693988, "alias": "Geordie", "name": "GeordieBunten", "user-since": datetime("2006-08-03T15:00:25.000Z"), "user-since-copy": datetime("2006-08-03T15:00:25.000Z"), "friend-ids": {{ 31987089, 15556815, 3656365, 35713356, 9573642, 38459850, 44400137, 44882118, 44921684, 47393814, 7869122, 35085016, 43725704, 17602789, 9966406, 20936803, 26425879, 41666932 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2007-01-20"), "end-date": null } ] }
+{ "id": 9696160, "id-copy": 9696160, "alias": "Lawerence", "name": "LawerenceLudwig", "user-since": datetime("2005-09-04T07:08:01.000Z"), "user-since-copy": datetime("2005-09-04T07:08:01.000Z"), "friend-ids": {{ 33125788, 14719007, 35434564 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2001-02-02"), "end-date": null } ] }
+{ "id": 9719995, "id-copy": 9719995, "alias": "Hazel", "name": "HazelKnopsnider", "user-since": datetime("2007-04-05T01:11:42.000Z"), "user-since-copy": datetime("2007-04-05T01:11:42.000Z"), "friend-ids": {{ 38515770, 23212874, 6000594, 27957554, 28093880, 3726628, 22800428, 42313894, 23190476, 18537188, 22083915, 43478674, 33364444, 19158958, 1590605, 36792931, 42057988, 33286729, 29580197, 25232028 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2008-09-28"), "end-date": null } ] }
+{ "id": 9736855, "id-copy": 9736855, "alias": "Sudie", "name": "SudieAlbright", "user-since": datetime("2011-10-08T08:46:27.000Z"), "user-since-copy": datetime("2011-10-08T08:46:27.000Z"), "friend-ids": {{ 20506190, 13537252, 46211902, 4320089 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2004-12-07"), "end-date": date("2010-07-02") } ] }
+{ "id": 9744016, "id-copy": 9744016, "alias": "Kasha", "name": "KashaMueller", "user-since": datetime("2011-03-16T17:17:31.000Z"), "user-since-copy": datetime("2011-03-16T17:17:31.000Z"), "friend-ids": {{ 15857660, 46791109, 10310040, 42863950, 19533508, 32561502, 4367358, 31952243, 7130063, 19536081, 19870534, 3642001, 910385, 28668446, 33204842, 13210089, 2805429 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2000-11-01"), "end-date": null } ] }
+{ "id": 9746482, "id-copy": 9746482, "alias": "Ava", "name": "AvaEndsley", "user-since": datetime("2005-07-05T11:34:59.000Z"), "user-since-copy": datetime("2005-07-05T11:34:59.000Z"), "friend-ids": {{ 38589612, 37168849, 27697487, 47869699, 7140447, 1195276, 25105593, 46071, 5222989, 39550451, 45838187, 8513498, 44093597, 25194162, 11534580, 37101502, 6417166, 23315276, 9854625 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2011-06-15"), "end-date": null } ] }
+{ "id": 9748939, "id-copy": 9748939, "alias": "April", "name": "AprilCourtney", "user-since": datetime("2008-02-10T17:35:28.000Z"), "user-since-copy": datetime("2008-02-10T17:35:28.000Z"), "friend-ids": {{ 43018591, 38860193, 26524230, 23704979, 2293321, 18201469, 41569073, 26942967, 16348102, 20218840, 30888146, 7584389, 11355443, 3703344 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2001-08-26"), "end-date": null } ] }
+{ "id": 9760834, "id-copy": 9760834, "alias": "Lavette", "name": "LavettePirl", "user-since": datetime("2006-02-12T07:28:53.000Z"), "user-since-copy": datetime("2006-02-12T07:28:53.000Z"), "friend-ids": {{ 27450797, 36415787 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2002-09-20"), "end-date": null } ] }
+{ "id": 9761152, "id-copy": 9761152, "alias": "Royle", "name": "RoyleStewart", "user-since": datetime("2010-05-15T17:14:18.000Z"), "user-since-copy": datetime("2010-05-15T17:14:18.000Z"), "friend-ids": {{ 21868661, 15545005, 11285872, 45768523, 12486235 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2002-11-20"), "end-date": null } ] }
+{ "id": 9767755, "id-copy": 9767755, "alias": "Joel", "name": "JoelHoopengarner", "user-since": datetime("2012-01-19T13:22:46.000Z"), "user-since-copy": datetime("2012-01-19T13:22:46.000Z"), "friend-ids": {{ 41934568, 20874721, 33807743 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2012-01-21"), "end-date": date("2012-06-09") } ] }
+{ "id": 9784687, "id-copy": 9784687, "alias": "Larrie", "name": "LarrieStroh", "user-since": datetime("2005-12-03T13:45:30.000Z"), "user-since-copy": datetime("2005-12-03T13:45:30.000Z"), "friend-ids": {{ 38055237, 43436653, 21194063, 30405058, 7754813, 14616686, 3434657, 24778389, 5653770, 8600235, 44560871, 4379727, 32140404, 35445864, 24133933, 21379278, 45626842, 25710375, 25970333, 16831917 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2006-09-18"), "end-date": null } ] }
+{ "id": 9795463, "id-copy": 9795463, "alias": "Brunilda", "name": "BrunildaPheleps", "user-since": datetime("2007-04-21T01:56:02.000Z"), "user-since-copy": datetime("2007-04-21T01:56:02.000Z"), "friend-ids": {{ 39507879, 43296507, 45019669, 39481546, 16657717, 8707249, 47148318, 46560087, 42473978, 11974026, 40145543, 2127794, 19537942, 28159963, 21439105, 32578039, 24112998, 47853039, 6406099, 30697429 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2001-07-13"), "end-date": null } ] }
+{ "id": 9805759, "id-copy": 9805759, "alias": "Emmie", "name": "EmmieJohns", "user-since": datetime("2008-11-01T15:15:13.000Z"), "user-since-copy": datetime("2008-11-01T15:15:13.000Z"), "friend-ids": {{ 47090234, 24484835, 11048702 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2008-02-26"), "end-date": null } ] }
+{ "id": 9809977, "id-copy": 9809977, "alias": "Kassandra", "name": "KassandraHarding", "user-since": datetime("2007-05-01T06:22:22.000Z"), "user-since-copy": datetime("2007-05-01T06:22:22.000Z"), "friend-ids": {{ 29945374, 38811992, 41372042, 28714909, 16897620, 5020268, 24134801, 26310926, 32871167, 18787983, 47295432, 31873694, 36300817, 42779931, 27486692 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-08-26"), "end-date": null } ] }
+{ "id": 9818617, "id-copy": 9818617, "alias": "Elwyn", "name": "ElwynEndsley", "user-since": datetime("2012-04-12T18:14:54.000Z"), "user-since-copy": datetime("2012-04-12T18:14:54.000Z"), "friend-ids": {{ 44007613, 15744997, 9366576, 44776374, 19082361, 9967101, 25247773, 20407697 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2003-09-09"), "end-date": null } ] }
+{ "id": 9842389, "id-copy": 9842389, "alias": "Nicolas", "name": "NicolasHynes", "user-since": datetime("2005-08-10T23:35:18.000Z"), "user-since-copy": datetime("2005-08-10T23:35:18.000Z"), "friend-ids": {{ 40180500, 33396487, 26907885, 4321366, 10229201, 41118923 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2006-10-23"), "end-date": date("2010-03-11") } ] }
+{ "id": 9856990, "id-copy": 9856990, "alias": "Claud", "name": "ClaudBaird", "user-since": datetime("2006-10-10T11:48:09.000Z"), "user-since-copy": datetime("2006-10-10T11:48:09.000Z"), "friend-ids": {{ 41756695, 15842897, 29797715, 13771892, 21179308, 42974840, 22223660, 35004748, 35597685, 45300254, 31116834, 42699991, 9704157, 23181215, 14806554, 8198556, 16256974, 16360634, 34736641 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2008-07-23"), "end-date": null } ] }
+{ "id": 9872791, "id-copy": 9872791, "alias": "Yasmine", "name": "YasmineCanham", "user-since": datetime("2005-06-08T14:45:42.000Z"), "user-since-copy": datetime("2005-06-08T14:45:42.000Z"), "friend-ids": {{ 7340569, 16137560, 43341029, 31700386, 24881875, 17852264, 42730676, 32655012 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2004-05-09"), "end-date": date("2011-02-28") } ] }
+{ "id": 9877837, "id-copy": 9877837, "alias": "Marilee", "name": "MarileeDowning", "user-since": datetime("2007-09-06T15:02:25.000Z"), "user-since-copy": datetime("2007-09-06T15:02:25.000Z"), "friend-ids": {{ 3032720, 7000379, 16658012, 33487490, 624779, 13480315, 8308906, 6949934, 9472007, 36568244, 41737195, 1310478, 42870077, 46663613 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2005-10-09"), "end-date": null } ] }
+{ "id": 9929866, "id-copy": 9929866, "alias": "Emilie", "name": "EmilieJohns", "user-since": datetime("2009-10-01T00:51:03.000Z"), "user-since-copy": datetime("2009-10-01T00:51:03.000Z"), "friend-ids": {{ 45496950, 38109555, 46259676, 14141368, 31720484, 35564907, 23226721, 36026226, 34003258, 47176035, 46593035, 5050811, 27858647, 3784968 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2012-08-10"), "end-date": date("2012-08-24") } ] }
+{ "id": 9937957, "id-copy": 9937957, "alias": "Corey", "name": "CoreyTaggart", "user-since": datetime("2005-11-25T16:13:03.000Z"), "user-since-copy": datetime("2005-11-25T16:13:03.000Z"), "friend-ids": {{ 40105038, 9364511, 47362703, 1876955, 3505769, 41708385, 36179634, 7022850 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2005-09-24"), "end-date": null } ] }
+{ "id": 9939937, "id-copy": 9939937, "alias": "Margeret", "name": "MargeretWhite", "user-since": datetime("2008-10-10T22:07:17.000Z"), "user-since-copy": datetime("2008-10-10T22:07:17.000Z"), "friend-ids": {{ 12369844, 34252449, 12412010, 16942281, 25231122, 42326296, 27054531, 8338820, 25466132, 10175756, 23763550, 40035149, 41030740, 36493305, 19615682, 30813330, 24869907, 6934392, 31309446, 2545800, 463498, 3089623, 12714051, 38317605 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2012-01-19"), "end-date": null } ] }
+{ "id": 9950824, "id-copy": 9950824, "alias": "Maryann", "name": "MaryannCressman", "user-since": datetime("2011-02-25T17:51:21.000Z"), "user-since-copy": datetime("2011-02-25T17:51:21.000Z"), "friend-ids": {{ 30203965, 23348792, 19093409, 21079475 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2006-10-11"), "end-date": date("2006-10-09") } ] }
+{ "id": 9955486, "id-copy": 9955486, "alias": "Jerrod", "name": "JerrodBeach", "user-since": datetime("2007-04-18T07:24:36.000Z"), "user-since-copy": datetime("2007-04-18T07:24:36.000Z"), "friend-ids": {{ 9760902, 36268051, 11373781, 42337286, 41818514, 20451257, 23673069, 14313303, 6548991, 34820597, 17346574, 46871090, 263833, 38179383, 14434022 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2003-09-01"), "end-date": date("2007-06-11") } ] }
+{ "id": 9968869, "id-copy": 9968869, "alias": "Shemika", "name": "ShemikaNickolson", "user-since": datetime("2005-02-20T10:34:04.000Z"), "user-since-copy": datetime("2005-02-20T10:34:04.000Z"), "friend-ids": {{ 30287118, 877645, 9968776, 31800907 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2009-07-23"), "end-date": null } ] }
+{ "id": 9978190, "id-copy": 9978190, "alias": "Tatianna", "name": "TatiannaSchmidt", "user-since": datetime("2012-07-05T14:37:56.000Z"), "user-since-copy": datetime("2012-07-05T14:37:56.000Z"), "friend-ids": {{ 15128198 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2008-11-17"), "end-date": null } ] }
+{ "id": 10026061, "id-copy": 10026061, "alias": "Nonie", "name": "NonieChappel", "user-since": datetime("2007-06-22T10:06:38.000Z"), "user-since-copy": datetime("2007-06-22T10:06:38.000Z"), "friend-ids": {{ 38760716, 16809503, 6592849, 3736630, 32388289, 40487693, 27146403, 22621793, 35615399, 10839746, 693037, 25222841, 46448329, 40740448, 21652202, 30069817, 21957966 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2010-08-19"), "end-date": date("2010-08-17") } ] }
+{ "id": 10045915, "id-copy": 10045915, "alias": "Mona", "name": "MonaMarshall", "user-since": datetime("2005-08-24T06:03:43.000Z"), "user-since-copy": datetime("2005-08-24T06:03:43.000Z"), "friend-ids": {{ 34157870, 1960568, 39038094, 2842182, 12353591, 44464974, 45836337, 4831806, 18179039, 21060089, 15776264, 41865218, 5999176, 18197780 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2006-07-28"), "end-date": null } ] }
+{ "id": 10069987, "id-copy": 10069987, "alias": "Andrina", "name": "AndrinaFisher", "user-since": datetime("2012-07-21T07:28:30.000Z"), "user-since-copy": datetime("2012-07-21T07:28:30.000Z"), "friend-ids": {{ 42024943, 39627436, 28414443, 36703363, 45477433, 37499278, 28548620, 6687009, 22700392, 47812034, 16805789, 33222895, 36328879, 20191886, 32457353, 14008353 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2004-12-11"), "end-date": date("2004-09-07") } ] }
+{ "id": 10073632, "id-copy": 10073632, "alias": "Hadley", "name": "HadleyPainter", "user-since": datetime("2010-08-18T16:57:45.000Z"), "user-since-copy": datetime("2010-08-18T16:57:45.000Z"), "friend-ids": {{ 35310707, 40074121, 28614727, 29388510, 29966750, 45475518, 5989395, 9892960, 7137969, 5530675, 2278234, 9571067, 29644726, 30689189, 41083149 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2004-06-13"), "end-date": date("2004-11-28") } ] }
+{ "id": 10087876, "id-copy": 10087876, "alias": "Carlyle", "name": "CarlyleMoberly", "user-since": datetime("2009-09-12T03:44:36.000Z"), "user-since-copy": datetime("2009-09-12T03:44:36.000Z"), "friend-ids": {{ 22254101, 16994379, 42146906, 28928982 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2012-07-24"), "end-date": date("2012-07-09") } ] }
+{ "id": 10108534, "id-copy": 10108534, "alias": "Moriah", "name": "MoriahMitchell", "user-since": datetime("2005-11-13T21:32:41.000Z"), "user-since-copy": datetime("2005-11-13T21:32:41.000Z"), "friend-ids": {{ 30372632 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2012-05-07"), "end-date": null } ] }
+{ "id": 10118077, "id-copy": 10118077, "alias": "Elizbeth", "name": "ElizbethPfeifer", "user-since": datetime("2011-09-08T11:58:48.000Z"), "user-since-copy": datetime("2011-09-08T11:58:48.000Z"), "friend-ids": {{ 18001251, 40309720, 10119557, 37766102, 22202316, 2805709, 693628, 5524288, 21415560, 45687644, 23912525, 25418741, 22816155, 26787291, 30518473, 27701649 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2002-03-15"), "end-date": date("2004-11-03") } ] }
+{ "id": 10126408, "id-copy": 10126408, "alias": "Pen", "name": "PenFleming", "user-since": datetime("2005-11-11T08:50:34.000Z"), "user-since-copy": datetime("2005-11-11T08:50:34.000Z"), "friend-ids": {{ 38072630, 45021886, 23988042, 41084533, 4743969, 7223979, 19120365, 44219284, 4691449, 21072839, 32536521, 36335527, 47376347, 16882811, 43140173, 7610811, 28217191, 25488874, 27968660, 13102347, 40169395, 25952056, 17249838, 30971677 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2011-10-04"), "end-date": date("2011-01-10") } ] }
+{ "id": 10131352, "id-copy": 10131352, "alias": "Brett", "name": "BrettBullard", "user-since": datetime("2011-03-20T00:21:15.000Z"), "user-since-copy": datetime("2011-03-20T00:21:15.000Z"), "friend-ids": {{ 42102691, 34313392, 19476509, 40509353, 40764048, 32856149, 20306336, 18276288, 34284082, 32265145, 23912229, 7426729, 26377621, 43687843, 6140857, 4573908, 6840657, 18335864, 19868141, 6051525 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2005-11-09"), "end-date": date("2008-12-05") } ] }
+{ "id": 10151953, "id-copy": 10151953, "alias": "Howard", "name": "HowardHoopengarner", "user-since": datetime("2006-07-23T01:43:57.000Z"), "user-since-copy": datetime("2006-07-23T01:43:57.000Z"), "friend-ids": {{ 32564548, 19333543, 27610653, 27936980, 7471201, 1353451, 30864511, 41582907, 22918030, 6011307, 21622284, 44695813, 34728110, 33062051, 29420834, 37472592, 3655974, 34618485, 21615748, 14107596, 15317302, 21805666, 4563480 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2012-06-08"), "end-date": null } ] }
+{ "id": 10162495, "id-copy": 10162495, "alias": "Malina", "name": "MalinaTrout", "user-since": datetime("2006-12-19T12:12:55.000Z"), "user-since-copy": datetime("2006-12-19T12:12:55.000Z"), "friend-ids": {{ 40578475, 43374248, 7059820, 18838227, 45149295, 47680877, 11640348, 19081155, 9959453, 46807478, 45192583, 39333999, 4869981, 42888726, 32789666, 19653202 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2000-11-08"), "end-date": null } ] }
+{ "id": 10177078, "id-copy": 10177078, "alias": "Fausto", "name": "FaustoLotherington", "user-since": datetime("2005-06-23T22:18:16.000Z"), "user-since-copy": datetime("2005-06-23T22:18:16.000Z"), "friend-ids": {{ 9405744, 13732034 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2007-12-27"), "end-date": null } ] }
+{ "id": 10178182, "id-copy": 10178182, "alias": "Jen", "name": "JenOtis", "user-since": datetime("2007-08-09T09:42:29.000Z"), "user-since-copy": datetime("2007-08-09T09:42:29.000Z"), "friend-ids": {{ 26278603, 27983753, 13714345, 35452213, 27849291, 21838200, 1008530, 27777115, 27069057, 35804914, 34598070, 10076890, 12795361, 16653787, 2916026, 27047674, 8630755, 29822673 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2005-10-10"), "end-date": null } ] }
+{ "id": 10179538, "id-copy": 10179538, "alias": "Orlando", "name": "OrlandoBaxter", "user-since": datetime("2006-02-06T08:33:07.000Z"), "user-since-copy": datetime("2006-02-06T08:33:07.000Z"), "friend-ids": {{ 6233497, 33888281, 44259464, 19279042, 22534429, 13084190, 38886041, 41675566, 3155617 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2009-07-06"), "end-date": null } ] }
+{ "id": 10186180, "id-copy": 10186180, "alias": "Mina", "name": "MinaGist", "user-since": datetime("2012-07-05T21:56:14.000Z"), "user-since-copy": datetime("2012-07-05T21:56:14.000Z"), "friend-ids": {{ 12424234, 41863508, 44607839, 36984124, 3839840, 38458170, 41721653, 4785194, 20595881, 13515001 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2012-07-19"), "end-date": null } ] }
+{ "id": 10190329, "id-copy": 10190329, "alias": "Rachyl", "name": "RachylAdams", "user-since": datetime("2005-08-25T14:09:48.000Z"), "user-since-copy": datetime("2005-08-25T14:09:48.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2002-11-17"), "end-date": null } ] }
+{ "id": 10193368, "id-copy": 10193368, "alias": "Oneida", "name": "OneidaEve", "user-since": datetime("2005-01-16T07:26:07.000Z"), "user-since-copy": datetime("2005-01-16T07:26:07.000Z"), "friend-ids": {{ 46396755, 39763353, 13661339, 5992749, 293256, 15572483, 16775625, 21543680 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2003-01-12"), "end-date": date("2008-03-22") } ] }
+{ "id": 10195063, "id-copy": 10195063, "alias": "Rose", "name": "RoseHatcher", "user-since": datetime("2008-10-11T02:17:54.000Z"), "user-since-copy": datetime("2008-10-11T02:17:54.000Z"), "friend-ids": {{ 9820231, 12294967, 46911959, 47936560, 7881400, 11585414, 45934029, 18009898, 11594812, 13760171, 41894550, 13254896, 28025170, 20007524, 13027888 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2010-03-26"), "end-date": null } ] }
+{ "id": 10197700, "id-copy": 10197700, "alias": "Frederica", "name": "FredericaCherry", "user-since": datetime("2006-04-10T01:23:53.000Z"), "user-since-copy": datetime("2006-04-10T01:23:53.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-01-01"), "end-date": date("2009-07-14") } ] }
+{ "id": 10207636, "id-copy": 10207636, "alias": "Stewart", "name": "StewartHamilton", "user-since": datetime("2008-11-06T21:44:47.000Z"), "user-since-copy": datetime("2008-11-06T21:44:47.000Z"), "friend-ids": {{ 25417411, 7322723, 13495699, 47274757, 44964322, 4993843, 36429109, 11904558, 18759232, 45446850, 40537858, 40487724, 36200691, 6846408, 7421262, 2225424, 12997194 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2012-08-28"), "end-date": date("2012-08-29") } ] }
+{ "id": 10215280, "id-copy": 10215280, "alias": "Barbara", "name": "BarbaraEve", "user-since": datetime("2012-03-09T01:36:52.000Z"), "user-since-copy": datetime("2012-03-09T01:36:52.000Z"), "friend-ids": {{ 32562793, 33679771, 10306498, 37847497, 30180151, 3504698 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2011-12-14"), "end-date": null } ] }
+{ "id": 10230604, "id-copy": 10230604, "alias": "Courtney", "name": "CourtneyCountryman", "user-since": datetime("2012-03-05T08:49:56.000Z"), "user-since-copy": datetime("2012-03-05T08:49:56.000Z"), "friend-ids": {{ 28617094, 31170285, 26700577, 43586990, 12809105, 8131401, 15644912, 38127923, 7871621, 13276397, 41863539, 3715524, 13404150, 12834697, 237361, 41295097, 29471386, 19859329, 14312407, 79917, 42547367, 9661712, 30110962, 29137807 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2001-06-09"), "end-date": date("2004-06-04") } ] }
+{ "id": 10251805, "id-copy": 10251805, "alias": "Jericho", "name": "JerichoBaird", "user-since": datetime("2005-07-02T12:57:18.000Z"), "user-since-copy": datetime("2005-07-02T12:57:18.000Z"), "friend-ids": {{ 5748549, 47013396, 15858292, 458526, 28324553, 22401875, 21726858, 38878600, 29844738, 14547049, 11432495, 9227475 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2005-06-09"), "end-date": date("2011-11-01") } ] }
+{ "id": 10261300, "id-copy": 10261300, "alias": "Nick", "name": "NickRohtin", "user-since": datetime("2007-01-24T17:56:52.000Z"), "user-since-copy": datetime("2007-01-24T17:56:52.000Z"), "friend-ids": {{ 37649902 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2004-03-06"), "end-date": date("2007-05-20") } ] }
+{ "id": 10267057, "id-copy": 10267057, "alias": "Thomas", "name": "ThomasCook", "user-since": datetime("2008-03-02T23:04:31.000Z"), "user-since-copy": datetime("2008-03-02T23:04:31.000Z"), "friend-ids": {{ 23744020, 25995598, 40459051, 27658275, 10133202, 11434833, 29790727, 1672639, 19652058, 18554997, 37878642, 48016133, 46599310, 37105777, 36004129, 6402365, 9889815, 29589019, 1497208, 19269802, 43383394, 30936085 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2008-05-14"), "end-date": date("2008-07-10") } ] }
+{ "id": 10278607, "id-copy": 10278607, "alias": "Brenden", "name": "BrendenLombardi", "user-since": datetime("2012-02-13T05:59:40.000Z"), "user-since-copy": datetime("2012-02-13T05:59:40.000Z"), "friend-ids": {{ 2820692, 43529738, 38518064, 29672334, 24653037, 39717291, 14213502, 23982828, 47123006, 34213620, 5993185, 10068793, 47512414, 40682283, 26631237, 23442819, 9215972, 9003752, 31259126, 8467245, 32821220, 8582002, 42606040 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2001-06-16"), "end-date": date("2008-09-11") } ] }
+{ "id": 10297336, "id-copy": 10297336, "alias": "Gayelord", "name": "GayelordCypret", "user-since": datetime("2005-09-28T10:01:31.000Z"), "user-since-copy": datetime("2005-09-28T10:01:31.000Z"), "friend-ids": {{ 43657472, 21189656, 43018991, 42333420, 27203617, 12389046, 44062328, 15441240, 31806533, 44999377, 30592890, 12304605, 6752099, 9488471, 5719065, 16290550, 23175098, 6432261 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-05-15"), "end-date": null } ] }
+{ "id": 10298530, "id-copy": 10298530, "alias": "Natalee", "name": "NataleeBell", "user-since": datetime("2010-09-07T14:14:59.000Z"), "user-since-copy": datetime("2010-09-07T14:14:59.000Z"), "friend-ids": {{ 36077399, 47946678, 4189158, 42122618, 14179077, 26433248, 25903252, 23116624, 33542934, 1071320, 31914369, 28408518, 40811454, 19212473, 25057330, 42758915 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2012-02-17"), "end-date": null } ] }
+{ "id": 10299298, "id-copy": 10299298, "alias": "Belinda", "name": "BelindaRockwell", "user-since": datetime("2005-03-08T07:13:05.000Z"), "user-since-copy": datetime("2005-03-08T07:13:05.000Z"), "friend-ids": {{ 31301282, 34653696, 23868758 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2008-08-19"), "end-date": null } ] }
+{ "id": 10300027, "id-copy": 10300027, "alias": "Cassie", "name": "CassieCarmichael", "user-since": datetime("2007-02-17T16:12:21.000Z"), "user-since-copy": datetime("2007-02-17T16:12:21.000Z"), "friend-ids": {{ 18690821, 9246387, 5425670, 8058755, 32156367, 29092478 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2000-03-06"), "end-date": null } ] }
+{ "id": 10338907, "id-copy": 10338907, "alias": "Leah", "name": "LeahStroble", "user-since": datetime("2010-12-07T08:23:00.000Z"), "user-since-copy": datetime("2010-12-07T08:23:00.000Z"), "friend-ids": {{ 25263375, 47112518, 47910837, 14446727, 35708710, 41365949, 8534511, 34992353, 1706302, 21380997, 47197876, 29441929, 4157771, 8674755, 14520863, 22041433, 47176591, 4072306, 47354501 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2003-03-25"), "end-date": null } ] }
+{ "id": 10390954, "id-copy": 10390954, "alias": "Lucinda", "name": "LucindaWatson", "user-since": datetime("2006-11-16T21:20:41.000Z"), "user-since-copy": datetime("2006-11-16T21:20:41.000Z"), "friend-ids": {{ 36017573, 9298650, 16054222, 21985420, 23378246, 30163820, 20942039, 28917630, 20851877, 41794807, 45887537, 39768986, 42476881, 5070921, 29487760, 24953551, 32065985, 16342096, 41522555, 41923127, 34675252, 10040601, 32604114, 23852658 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2009-03-14"), "end-date": null } ] }
+{ "id": 10391077, "id-copy": 10391077, "alias": "Tracy", "name": "TracyHiles", "user-since": datetime("2005-11-19T21:08:51.000Z"), "user-since-copy": datetime("2005-11-19T21:08:51.000Z"), "friend-ids": {{ 27119048, 1983772, 38766385, 35631268, 14736954, 7586158, 45840742, 27211063, 33946244, 1590669, 22363833, 19668917, 12778790, 31993728, 4498870, 68121, 13591025, 13285639 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2012-07-12"), "end-date": null } ] }
+{ "id": 10394488, "id-copy": 10394488, "alias": "Oswald", "name": "OswaldRay", "user-since": datetime("2006-02-12T17:39:23.000Z"), "user-since-copy": datetime("2006-02-12T17:39:23.000Z"), "friend-ids": {{ 14370372, 14174983, 7749259, 39375970, 1755409, 9056913 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2011-12-04"), "end-date": date("2011-06-08") } ] }
+{ "id": 10396831, "id-copy": 10396831, "alias": "Carman", "name": "CarmanElder", "user-since": datetime("2011-12-27T21:50:41.000Z"), "user-since-copy": datetime("2011-12-27T21:50:41.000Z"), "friend-ids": {{ 41782166, 39862540, 39100006, 45023958, 29253172, 31208143, 12637805, 5844876, 37296616, 20896053, 18358082, 11068853, 5350064, 14456765, 15758928 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2012-05-18"), "end-date": date("2012-07-26") } ] }
+{ "id": 10397017, "id-copy": 10397017, "alias": "Holly", "name": "HollyHatch", "user-since": datetime("2006-04-12T03:26:11.000Z"), "user-since-copy": datetime("2006-04-12T03:26:11.000Z"), "friend-ids": {{ 1504006, 21411501, 20934982, 24019384, 8634101, 25659178, 16581112, 2481631, 15544800 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2000-12-04"), "end-date": null } ] }
+{ "id": 10404706, "id-copy": 10404706, "alias": "Rylan", "name": "RylanEmrick", "user-since": datetime("2008-11-23T00:55:36.000Z"), "user-since-copy": datetime("2008-11-23T00:55:36.000Z"), "friend-ids": {{ 17936230, 20908773, 34834317, 26134774, 3534090, 7699389, 11743997, 37809096, 23228338, 19069026, 662582, 40839640, 26706968, 42711557, 28658968, 39161015, 29201879, 7516443, 21802464, 16456657, 32689464 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2012-02-16"), "end-date": null } ] }
+{ "id": 10413097, "id-copy": 10413097, "alias": "Lindsay", "name": "LindsayDoverspike", "user-since": datetime("2005-03-24T22:42:49.000Z"), "user-since-copy": datetime("2005-03-24T22:42:49.000Z"), "friend-ids": {{ 773762, 43764188, 23133486, 27099138, 38010544, 38283504, 38432745, 32450505, 34499948, 38200436, 44093983, 41684052, 41353940, 29027114, 2947798, 25212070, 9522627, 18680730, 13060818, 41586559 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2010-01-23"), "end-date": date("2011-01-14") } ] }
+{ "id": 10422310, "id-copy": 10422310, "alias": "Edmundo", "name": "EdmundoShaw", "user-since": datetime("2012-07-02T11:10:15.000Z"), "user-since-copy": datetime("2012-07-02T11:10:15.000Z"), "friend-ids": {{ 4235436, 16381036, 12579129, 43280339, 16455681, 28445764, 10796826, 28577255, 15173785, 47982248, 11990921, 2093558, 6244669, 4830927, 34859603, 22246754, 45142656 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2011-01-27"), "end-date": null } ] }
+{ "id": 10423588, "id-copy": 10423588, "alias": "Shirlene", "name": "ShirleneRuch", "user-since": datetime("2006-04-09T05:52:24.000Z"), "user-since-copy": datetime("2006-04-09T05:52:24.000Z"), "friend-ids": {{ 15418780, 12724265, 27282306, 13592995, 24753166, 32824252, 40619106, 27563604, 12337625, 45387219, 27749581, 44912564, 37470078, 19663516 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2003-06-17"), "end-date": null } ] }
+{ "id": 10469071, "id-copy": 10469071, "alias": "Apryl", "name": "AprylWatson", "user-since": datetime("2006-10-03T08:37:12.000Z"), "user-since-copy": datetime("2006-10-03T08:37:12.000Z"), "friend-ids": {{ 4517575, 34635569, 1199146 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2001-02-01"), "end-date": date("2007-09-01") } ] }
+{ "id": 10472248, "id-copy": 10472248, "alias": "Harry", "name": "HarryDugmore", "user-since": datetime("2012-02-18T05:46:12.000Z"), "user-since-copy": datetime("2012-02-18T05:46:12.000Z"), "friend-ids": {{ 30193978, 30762534, 24660208, 29628319, 30687391, 39795396, 33525293, 23739628, 28969085, 30275276, 3497701, 17091988, 15259527, 25164171, 34052417, 4318314, 1876063, 29984074, 3421436, 16610126 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2012-01-19"), "end-date": date("2012-01-02") } ] }
+{ "id": 10478512, "id-copy": 10478512, "alias": "Remona", "name": "RemonaPittman", "user-since": datetime("2007-06-19T12:20:07.000Z"), "user-since-copy": datetime("2007-06-19T12:20:07.000Z"), "friend-ids": {{ 12750727 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2001-02-02"), "end-date": null } ] }
+{ "id": 10484578, "id-copy": 10484578, "alias": "Troy", "name": "TroyWheeler", "user-since": datetime("2006-12-19T11:23:18.000Z"), "user-since-copy": datetime("2006-12-19T11:23:18.000Z"), "friend-ids": {{ 13536585, 23059550, 16602050, 12025612, 25014410, 13465266 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2011-10-23"), "end-date": null } ] }
+{ "id": 10492168, "id-copy": 10492168, "alias": "Savannah", "name": "SavannahRobinson", "user-since": datetime("2008-05-02T04:19:01.000Z"), "user-since-copy": datetime("2008-05-02T04:19:01.000Z"), "friend-ids": {{ 40126719, 38171650, 1474355, 6983398, 7918678, 45578368, 3210188, 29374863, 37758187, 2415003, 13746140, 44168763, 45798029, 17203664, 46309082, 21338452, 17217009, 24916114 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2009-07-20"), "end-date": date("2009-03-01") } ] }
+{ "id": 10495420, "id-copy": 10495420, "alias": "Wendy", "name": "WendyMcloskey", "user-since": datetime("2011-04-26T23:38:24.000Z"), "user-since-copy": datetime("2011-04-26T23:38:24.000Z"), "friend-ids": {{ 16762653, 46262691, 12313140, 20481262, 347993, 23105127, 1680519, 20880265, 45611347, 21907223, 46615281, 17188244, 44019800, 46943250, 28647738, 16792673, 29406270, 42714079 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2008-08-27"), "end-date": null } ] }
+{ "id": 10498285, "id-copy": 10498285, "alias": "Kiley", "name": "KileyBridger", "user-since": datetime("2006-05-14T21:55:34.000Z"), "user-since-copy": datetime("2006-05-14T21:55:34.000Z"), "friend-ids": {{ 38780484, 46190003, 905670, 35609390, 46621151, 5099226, 24328595, 16340411, 13326485, 13872400, 35896828, 9196151, 8525875, 7461206, 28379538, 46461267, 45270205, 35718577, 5310596, 7080391 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2009-11-11"), "end-date": date("2009-06-23") } ] }
+{ "id": 10501429, "id-copy": 10501429, "alias": "Danielle", "name": "DanielleYoung", "user-since": datetime("2010-04-24T05:46:06.000Z"), "user-since-copy": datetime("2010-04-24T05:46:06.000Z"), "friend-ids": {{ 7960737, 27505427 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2002-07-27"), "end-date": date("2004-07-28") } ] }
+{ "id": 10504084, "id-copy": 10504084, "alias": "Etsuko", "name": "EtsukoDealtry", "user-since": datetime("2012-05-11T00:35:22.000Z"), "user-since-copy": datetime("2012-05-11T00:35:22.000Z"), "friend-ids": {{ 27578969, 40308832, 15379566, 8664135, 21276773, 43659426, 28027401, 23264043, 23981731, 19124540, 36281456, 38766688, 37886842, 20522702, 28559857, 9838362, 30409517, 14237008, 41013610, 41586760, 37285778, 29427060, 45678692, 32255048 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2011-10-12"), "end-date": date("2011-12-04") } ] }
+{ "id": 10505419, "id-copy": 10505419, "alias": "Anderson", "name": "AndersonSoames", "user-since": datetime("2009-04-01T01:24:07.000Z"), "user-since-copy": datetime("2009-04-01T01:24:07.000Z"), "friend-ids": {{ 25420744, 34012676, 8558565, 45471514, 12117008, 35275, 4952379, 46480100, 29394067, 15504329, 18153717, 8476606, 19867236, 35743164, 38523474, 6479207, 31151752, 19687338, 5379846, 32574974, 26920356 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2005-08-01"), "end-date": null } ] }
+{ "id": 10514095, "id-copy": 10514095, "alias": "Chantelle", "name": "ChantelleCatleay", "user-since": datetime("2008-10-23T00:05:15.000Z"), "user-since-copy": datetime("2008-10-23T00:05:15.000Z"), "friend-ids": {{ 11871759, 1505524, 45483061, 31479407, 15112731, 41816114, 22650998 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2008-06-14"), "end-date": null } ] }
+{ "id": 10514428, "id-copy": 10514428, "alias": "Eliseo", "name": "EliseoHoffhants", "user-since": datetime("2012-08-24T08:40:51.000Z"), "user-since-copy": datetime("2012-08-24T08:40:51.000Z"), "friend-ids": {{ 45751891, 26026786, 24531389, 26239368, 34021241 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2010-03-01"), "end-date": date("2010-08-02") } ] }
+{ "id": 10515721, "id-copy": 10515721, "alias": "Mariano", "name": "MarianoTrout", "user-since": datetime("2007-08-27T09:33:28.000Z"), "user-since-copy": datetime("2007-08-27T09:33:28.000Z"), "friend-ids": {{ 18516004, 4847094, 31548989, 28302698, 18308169, 15068883, 33358074, 19739053, 34017693 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2004-10-18"), "end-date": null } ] }
+{ "id": 10547020, "id-copy": 10547020, "alias": "Reita", "name": "ReitaBlunt", "user-since": datetime("2006-01-18T16:51:49.000Z"), "user-since-copy": datetime("2006-01-18T16:51:49.000Z"), "friend-ids": {{ 34373903, 36464697, 37171525, 19138424, 24675436, 16269152, 43940985, 2735762, 32760257, 42561749, 45516984, 39110107, 21610913, 1805884, 3342035, 40703512, 11665984, 29345992, 41497492, 30054924, 18098215 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-12-01"), "end-date": null } ] }
+{ "id": 10548142, "id-copy": 10548142, "alias": "Dannie", "name": "DannieTillson", "user-since": datetime("2007-03-07T04:57:23.000Z"), "user-since-copy": datetime("2007-03-07T04:57:23.000Z"), "friend-ids": {{ 37443492, 21615683, 5655492, 24162015, 46418787, 46328489, 26669127, 38324141 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2012-01-03"), "end-date": null } ] }
+{ "id": 10554112, "id-copy": 10554112, "alias": "Virgil", "name": "VirgilBickerson", "user-since": datetime("2006-03-14T07:07:42.000Z"), "user-since-copy": datetime("2006-03-14T07:07:42.000Z"), "friend-ids": {{ 21584501, 3506050, 31062036, 20425233, 6548274, 12613206, 16607156 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2004-08-25"), "end-date": date("2006-11-11") } ] }
+{ "id": 10567702, "id-copy": 10567702, "alias": "Zelda", "name": "ZeldaRitter", "user-since": datetime("2010-09-27T12:52:54.000Z"), "user-since-copy": datetime("2010-09-27T12:52:54.000Z"), "friend-ids": {{ 28336161, 20248788, 24723848, 8856879, 16831898, 7643547, 42868543, 23023606, 7531861, 36186817, 29113040, 995506, 32607538, 18755505, 44683178, 24627205, 39736850, 43535271, 385416, 40525568 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2011-11-27"), "end-date": date("2011-08-16") } ] }
+{ "id": 10585294, "id-copy": 10585294, "alias": "Bryan", "name": "BryanEliza", "user-since": datetime("2005-02-03T16:20:19.000Z"), "user-since-copy": datetime("2005-02-03T16:20:19.000Z"), "friend-ids": {{ 6407647, 24838863, 45997254, 42728806, 37001718, 46932382 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2003-03-15"), "end-date": date("2008-04-24") } ] }
+{ "id": 10594069, "id-copy": 10594069, "alias": "Clinton", "name": "ClintonMiller", "user-since": datetime("2007-03-12T05:19:19.000Z"), "user-since-copy": datetime("2007-03-12T05:19:19.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "itlab", "start-date": date("2010-06-06"), "end-date": null } ] }
+{ "id": 10636498, "id-copy": 10636498, "alias": "Grahame", "name": "GrahameLeslie", "user-since": datetime("2006-01-17T16:17:07.000Z"), "user-since-copy": datetime("2006-01-17T16:17:07.000Z"), "friend-ids": {{ 3924169, 14543253, 19830425, 34696361, 26630699, 47664771 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2004-03-25"), "end-date": null } ] }
+{ "id": 10650526, "id-copy": 10650526, "alias": "Gertie", "name": "GertieWallace", "user-since": datetime("2010-07-16T05:33:07.000Z"), "user-since-copy": datetime("2010-07-16T05:33:07.000Z"), "friend-ids": {{ 35934417, 43053648, 35859770, 43704932, 35605486, 17212020, 21235775, 26783725, 17450538, 42996452, 15873053, 36331217, 18524993, 45483950, 1549676, 24801562, 46527491 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2003-08-16"), "end-date": null } ] }
+{ "id": 10655089, "id-copy": 10655089, "alias": "Quinn", "name": "QuinnHays", "user-since": datetime("2009-11-25T04:42:39.000Z"), "user-since-copy": datetime("2009-11-25T04:42:39.000Z"), "friend-ids": {{ 17385636, 24378500, 37614592, 32315940, 18046144, 45823175, 29709981, 28423306, 23783823, 10623867, 27782698 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2012-06-09"), "end-date": null } ] }
+{ "id": 10659022, "id-copy": 10659022, "alias": "Cecelia", "name": "CeceliaHandyside", "user-since": datetime("2007-02-22T12:42:30.000Z"), "user-since-copy": datetime("2007-02-22T12:42:30.000Z"), "friend-ids": {{ 9051, 38746030, 6178049, 22068473, 25755202, 11577837, 28994476 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2004-07-09"), "end-date": date("2009-10-14") } ] }
+{ "id": 10674199, "id-copy": 10674199, "alias": "Dorothy", "name": "DorothyPritchard", "user-since": datetime("2007-09-19T04:32:05.000Z"), "user-since-copy": datetime("2007-09-19T04:32:05.000Z"), "friend-ids": {{ 11239155, 14468542, 8244419, 30563447, 2235193, 33015958, 11941749, 22198664, 41531114, 11614864, 43486312, 11394784, 46038310, 8248070, 12346192 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2000-10-03"), "end-date": null } ] }
+{ "id": 10701727, "id-copy": 10701727, "alias": "Paulita", "name": "PaulitaHays", "user-since": datetime("2009-11-15T15:25:08.000Z"), "user-since-copy": datetime("2009-11-15T15:25:08.000Z"), "friend-ids": {{ 31869253, 13336594, 19116516, 30920596 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2001-12-10"), "end-date": null } ] }
+{ "id": 10712731, "id-copy": 10712731, "alias": "Abigail", "name": "AbigailKunkle", "user-since": datetime("2011-07-20T07:10:43.000Z"), "user-since-copy": datetime("2011-07-20T07:10:43.000Z"), "friend-ids": {{ 35920648, 38798778, 17160209, 16674423, 44247736, 45731986, 29605307, 123608, 46926535, 41274265, 36397206, 16900492, 19895463, 10043680, 42549381, 21006240, 13037274, 25867242, 34428167, 953419, 2284340, 32321044, 2351589, 30797666 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2002-08-11"), "end-date": date("2002-12-01") } ] }
+{ "id": 10733305, "id-copy": 10733305, "alias": "Dakota", "name": "DakotaSmith", "user-since": datetime("2009-11-17T19:52:42.000Z"), "user-since-copy": datetime("2009-11-17T19:52:42.000Z"), "friend-ids": {{ 21984282, 14492326, 18724474, 17361116, 26773641, 32118673, 8295454, 6804824 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2007-05-28"), "end-date": null } ] }
+{ "id": 10739446, "id-copy": 10739446, "alias": "Urban", "name": "UrbanHair", "user-since": datetime("2010-12-28T02:29:19.000Z"), "user-since-copy": datetime("2010-12-28T02:29:19.000Z"), "friend-ids": {{ 31947556, 39058269, 43315882, 40575729, 4079275, 40689246, 22639555, 1422452, 28051313, 41854009, 30810426, 37406811, 20834349, 46933622, 28218698, 17239481, 33458180 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2006-05-21"), "end-date": null } ] }
+{ "id": 10742182, "id-copy": 10742182, "alias": "Tel", "name": "TelBowchiew", "user-since": datetime("2009-09-23T02:51:14.000Z"), "user-since-copy": datetime("2009-09-23T02:51:14.000Z"), "friend-ids": {{ 17515416, 42010238, 23580669, 26008148, 35744494 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2006-10-05"), "end-date": date("2007-05-26") } ] }
+{ "id": 10754107, "id-copy": 10754107, "alias": "Jeri", "name": "JeriSanner", "user-since": datetime("2009-11-15T23:47:08.000Z"), "user-since-copy": datetime("2009-11-15T23:47:08.000Z"), "friend-ids": {{ 19868241, 28778419, 16761189, 28588239, 1592484, 41256056, 36550491, 10555328, 3086612, 37431116, 45976270 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2004-11-06"), "end-date": null } ] }
+{ "id": 10760020, "id-copy": 10760020, "alias": "Emeline", "name": "EmelineCowher", "user-since": datetime("2006-03-11T07:02:10.000Z"), "user-since-copy": datetime("2006-03-11T07:02:10.000Z"), "friend-ids": {{ 2652618, 22247716, 39487944, 16288504, 8109009, 34390947, 2041892, 27800644, 5979423, 12674908 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2007-12-26"), "end-date": date("2007-09-04") } ] }
+{ "id": 10794448, "id-copy": 10794448, "alias": "Delmar", "name": "DelmarDowning", "user-since": datetime("2012-03-10T23:41:49.000Z"), "user-since-copy": datetime("2012-03-10T23:41:49.000Z"), "friend-ids": {{ 34002211, 41487, 45067426, 9754093, 23041928, 41378740, 4013550, 11584362, 46202858, 43273004, 35465505 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2005-09-12"), "end-date": null } ] }
+{ "id": 10808932, "id-copy": 10808932, "alias": "Sharita", "name": "SharitaGregory", "user-since": datetime("2006-09-17T04:48:23.000Z"), "user-since-copy": datetime("2006-09-17T04:48:23.000Z"), "friend-ids": {{ 41622567, 16559791, 6346693, 18540237, 14753253, 23252825, 17163196, 46962665, 26442426, 14344279, 17332246, 36154890, 22814241, 22709064, 32887290, 42853122, 23782934, 27425228, 22941847 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2008-06-08"), "end-date": date("2011-01-28") } ] }
+{ "id": 10809730, "id-copy": 10809730, "alias": "Algar", "name": "AlgarZaun", "user-since": datetime("2008-08-14T06:37:59.000Z"), "user-since-copy": datetime("2008-08-14T06:37:59.000Z"), "friend-ids": {{ 12676185, 26087426, 42241358, 47854149, 22179884, 34701736, 35541344, 46257087, 35091522, 10779069 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2010-09-13"), "end-date": null } ] }
+{ "id": 10851133, "id-copy": 10851133, "alias": "Wilbur", "name": "WilburDiegel", "user-since": datetime("2005-08-20T01:37:10.000Z"), "user-since-copy": datetime("2005-08-20T01:37:10.000Z"), "friend-ids": {{ 44811869, 15362002, 5320359, 4756538, 40097009, 905334, 44595717, 3685695, 35645656, 2090160, 35124514, 21715286, 26713020, 5816017, 15598653, 6425314, 10423130, 29593106, 14054734, 1780417, 38517315, 25570577, 5038946 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2004-05-04"), "end-date": null } ] }
+{ "id": 10867444, "id-copy": 10867444, "alias": "Tetty", "name": "TettyZundel", "user-since": datetime("2012-07-26T17:54:45.000Z"), "user-since-copy": datetime("2012-07-26T17:54:45.000Z"), "friend-ids": {{ 17830961, 13154371, 12005619, 15279158, 15766172, 3071670, 4314512, 29378453, 33264674, 32657723, 37875054, 6208013, 23310809, 11994927, 9787690, 25069760, 11104605, 44517542, 45829337, 26593992 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2001-09-25"), "end-date": null } ] }
+{ "id": 10874791, "id-copy": 10874791, "alias": "Haydee", "name": "HaydeeGarratt", "user-since": datetime("2007-04-14T00:19:00.000Z"), "user-since-copy": datetime("2007-04-14T00:19:00.000Z"), "friend-ids": {{ 12247794, 10306863, 33161811, 43877113, 37745696 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2008-03-07"), "end-date": date("2011-12-27") } ] }
+{ "id": 10892830, "id-copy": 10892830, "alias": "Audrie", "name": "AudrieHawkins", "user-since": datetime("2011-11-19T00:51:33.000Z"), "user-since-copy": datetime("2011-11-19T00:51:33.000Z"), "friend-ids": {{ 8838768, 18321840, 16958648, 27000957, 19090823, 11772058, 18573458, 24662627, 27415154, 4998699, 44522833, 44994903, 6514403, 43833807, 38512495, 6964420, 11334788, 14298721, 25316052, 11632302 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2005-02-04"), "end-date": null } ] }
+{ "id": 10902649, "id-copy": 10902649, "alias": "Makenzie", "name": "MakenzieWerner", "user-since": datetime("2005-12-20T00:23:45.000Z"), "user-since-copy": datetime("2005-12-20T00:23:45.000Z"), "friend-ids": {{ 9011568, 38173487, 45649445, 11873586 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2000-01-06"), "end-date": date("2009-03-24") } ] }
+{ "id": 10904125, "id-copy": 10904125, "alias": "Jarred", "name": "JarredRopes", "user-since": datetime("2005-11-09T09:53:06.000Z"), "user-since-copy": datetime("2005-11-09T09:53:06.000Z"), "friend-ids": {{ 26810, 23763346, 5064508, 26124598 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2007-12-28"), "end-date": date("2009-04-23") } ] }
+{ "id": 10911220, "id-copy": 10911220, "alias": "Laurice", "name": "LauriceDuncan", "user-since": datetime("2008-08-05T15:55:34.000Z"), "user-since-copy": datetime("2008-08-05T15:55:34.000Z"), "friend-ids": {{ 212109 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2001-02-03"), "end-date": null } ] }
+{ "id": 10911274, "id-copy": 10911274, "alias": "Bridgette", "name": "BridgetteBenford", "user-since": datetime("2007-02-15T06:18:45.000Z"), "user-since-copy": datetime("2007-02-15T06:18:45.000Z"), "friend-ids": {{ 10909520, 14433605 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2012-01-14"), "end-date": null } ] }
+{ "id": 10912441, "id-copy": 10912441, "alias": "Janae", "name": "JanaeErschoff", "user-since": datetime("2009-04-17T09:26:36.000Z"), "user-since-copy": datetime("2009-04-17T09:26:36.000Z"), "friend-ids": {{ 11445243, 13239218, 2302326, 37976140, 45374131, 14136536, 2051767, 7824391, 42808044, 41836900, 35275542, 33493951, 8497237, 42991362, 24049395, 32159562, 23378256, 4723574, 47010157 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2012-04-20"), "end-date": date("2012-04-04") } ] }
+{ "id": 10915261, "id-copy": 10915261, "alias": "Lyle", "name": "LyleMuller", "user-since": datetime("2010-10-16T16:36:46.000Z"), "user-since-copy": datetime("2010-10-16T16:36:46.000Z"), "friend-ids": {{ 28409003, 7495999, 10776059, 23825626, 44321306, 15679301, 36736470, 24070644, 14041140, 4784196, 19462533, 47300197, 33544003 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2006-09-25"), "end-date": null } ] }
+{ "id": 10930153, "id-copy": 10930153, "alias": "Liliana", "name": "LilianaGoodman", "user-since": datetime("2009-06-22T20:57:17.000Z"), "user-since-copy": datetime("2009-06-22T20:57:17.000Z"), "friend-ids": {{ 4302195, 1569986, 5108357, 40772631, 30372008, 36454077, 26878227, 10958250, 46069776, 4779188, 46627230, 47074148, 25489453, 24956443, 31679399, 21835639, 42097220, 35662047, 6354581, 34282348, 13473927 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2008-10-25"), "end-date": null } ] }
+{ "id": 10933138, "id-copy": 10933138, "alias": "Gwendoline", "name": "GwendolineCypret", "user-since": datetime("2006-04-10T03:55:29.000Z"), "user-since-copy": datetime("2006-04-10T03:55:29.000Z"), "friend-ids": {{ 9996028, 18756914, 15079751, 34129343, 44558538, 25387070, 44250368, 37560291, 5178625, 10379959, 39639296, 8784216, 13429736, 22802431, 11154064, 2453387, 24748342, 34032462, 32570963, 4861587, 19421488, 10848442 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2008-12-24"), "end-date": date("2010-05-20") } ] }
+{ "id": 10936273, "id-copy": 10936273, "alias": "Hans", "name": "HansMench", "user-since": datetime("2008-08-08T12:00:48.000Z"), "user-since-copy": datetime("2008-08-08T12:00:48.000Z"), "friend-ids": {{ 36800139 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2005-04-15"), "end-date": date("2009-08-05") } ] }
+{ "id": 10938328, "id-copy": 10938328, "alias": "Tyrese", "name": "TyreseStainforth", "user-since": datetime("2011-03-03T04:21:04.000Z"), "user-since-copy": datetime("2011-03-03T04:21:04.000Z"), "friend-ids": {{ 33557445, 27981614, 25595450, 31820772, 42028444, 31389097, 16332592, 3555278, 45113070, 5198333 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2000-06-04"), "end-date": null } ] }
+{ "id": 10943026, "id-copy": 10943026, "alias": "Raeburn", "name": "RaeburnAllshouse", "user-since": datetime("2008-08-26T04:51:27.000Z"), "user-since-copy": datetime("2008-08-26T04:51:27.000Z"), "friend-ids": {{ 6784667, 1651647, 45052591, 21630976, 20049039, 37839759, 38694475, 23340828, 8641638, 4568782, 35684305, 20895609, 2213341, 8612199, 14260231, 8621325, 21926952, 41656664, 45180955 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2007-09-28"), "end-date": null } ] }
+{ "id": 10957867, "id-copy": 10957867, "alias": "Zach", "name": "ZachOppenheimer", "user-since": datetime("2012-01-01T14:40:11.000Z"), "user-since-copy": datetime("2012-01-01T14:40:11.000Z"), "friend-ids": {{ 27759480, 2112389, 8560433, 10052851, 37714587, 16717012, 36648956, 44803993, 36030695, 5359496, 32302980, 27143894, 19287706 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2003-05-14"), "end-date": date("2004-02-23") } ] }
+{ "id": 10967254, "id-copy": 10967254, "alias": "Andre", "name": "AndreCowper", "user-since": datetime("2011-12-21T20:22:47.000Z"), "user-since-copy": datetime("2011-12-21T20:22:47.000Z"), "friend-ids": {{ 23645341, 16267661, 7660549, 24716202, 20945538, 10125828, 1712260, 5309070, 16802418, 18273953, 12670834 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2003-12-25"), "end-date": date("2004-04-09") } ] }
+{ "id": 10972447, "id-copy": 10972447, "alias": "Loretta", "name": "LorettaBriggs", "user-since": datetime("2005-07-01T10:25:33.000Z"), "user-since-copy": datetime("2005-07-01T10:25:33.000Z"), "friend-ids": {{ 6898813, 6606991, 14092255, 9865734, 23960698, 47354873, 19345256 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2005-06-02"), "end-date": null } ] }
+{ "id": 10989949, "id-copy": 10989949, "alias": "Kaylyn", "name": "KaylynElder", "user-since": datetime("2011-01-13T12:02:13.000Z"), "user-since-copy": datetime("2011-01-13T12:02:13.000Z"), "friend-ids": {{ 22698118, 31639011, 11500577, 13007617, 26781164, 20827141, 9916306, 26415081, 14027605, 19305199, 45276489, 17632806, 42243983 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2006-01-05"), "end-date": null } ] }
+{ "id": 10993267, "id-copy": 10993267, "alias": "Esmund", "name": "EsmundDunkle", "user-since": datetime("2005-11-16T21:18:20.000Z"), "user-since-copy": datetime("2005-11-16T21:18:20.000Z"), "friend-ids": {{ 1277480, 11393524, 32336542, 41857626, 7807437, 25280677, 17518254, 7723810, 18423045, 11937236, 21507800 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2006-12-03"), "end-date": date("2011-11-26") } ] }
+{ "id": 11001610, "id-copy": 11001610, "alias": "Keven", "name": "KevenWildman", "user-since": datetime("2006-09-07T02:21:33.000Z"), "user-since-copy": datetime("2006-09-07T02:21:33.000Z"), "friend-ids": {{ 14316856, 4291050 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2012-06-20"), "end-date": date("2012-06-09") } ] }
+{ "id": 11016043, "id-copy": 11016043, "alias": "Ellis", "name": "EllisVorrasi", "user-since": datetime("2009-08-26T16:43:17.000Z"), "user-since-copy": datetime("2009-08-26T16:43:17.000Z"), "friend-ids": {{ 41000811, 12639978, 14487796, 39651858, 40189282, 7834125, 44416511, 28673665 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2008-01-21"), "end-date": date("2008-04-26") } ] }
+{ "id": 11072782, "id-copy": 11072782, "alias": "Jewel", "name": "JewelSchreckengost", "user-since": datetime("2012-06-04T18:20:29.000Z"), "user-since-copy": datetime("2012-06-04T18:20:29.000Z"), "friend-ids": {{ 47896348, 34649239, 38135221, 19731900, 14383059, 3639686, 28133949, 1326525, 415048, 34486382, 32809579, 31754806, 33563370 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-03-06"), "end-date": null } ] }
+{ "id": 11087224, "id-copy": 11087224, "alias": "Zola", "name": "ZolaKnisely", "user-since": datetime("2005-11-18T05:30:00.000Z"), "user-since-copy": datetime("2005-11-18T05:30:00.000Z"), "friend-ids": {{ 6324130, 38065951, 14950455, 27869167, 32957819, 11157656, 10411400, 18072233, 35246039, 35345326, 23217009, 13495953, 18987122 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2003-05-04"), "end-date": null } ] }
+{ "id": 11111890, "id-copy": 11111890, "alias": "Geordie", "name": "GeordieGraff", "user-since": datetime("2006-02-12T04:30:44.000Z"), "user-since-copy": datetime("2006-02-12T04:30:44.000Z"), "friend-ids": {{ 12852237, 10391003, 37679153, 6620205, 25381043, 19805548, 4534765, 11626709, 47369482, 15045527, 25177819, 15113002, 39634176, 40637870, 47662386, 8045236 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2005-06-18"), "end-date": null } ] }
+{ "id": 11130781, "id-copy": 11130781, "alias": "Kenia", "name": "KeniaMiller", "user-since": datetime("2008-05-27T02:28:18.000Z"), "user-since-copy": datetime("2008-05-27T02:28:18.000Z"), "friend-ids": {{ 43139868, 16103105, 25352928, 23612973, 9645914, 20517323, 40438742, 47972276, 7395189, 44164898, 2805123, 33235701, 39846510, 21170026, 14223369, 14077979 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2011-06-24"), "end-date": date("2011-04-08") } ] }
+{ "id": 11131138, "id-copy": 11131138, "alias": "Maximillian", "name": "MaximillianSloan", "user-since": datetime("2009-12-26T13:02:42.000Z"), "user-since-copy": datetime("2009-12-26T13:02:42.000Z"), "friend-ids": {{ 4007900, 16474597, 36917058, 46709116, 35833748, 7074328, 6125321, 40646485, 23690629, 3251896, 3973740, 17863849, 9389737, 26501803, 4207105 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2010-10-16"), "end-date": null } ] }
+{ "id": 11135899, "id-copy": 11135899, "alias": "Bailey", "name": "BaileyMoonshower", "user-since": datetime("2011-08-28T07:36:28.000Z"), "user-since-copy": datetime("2011-08-28T07:36:28.000Z"), "friend-ids": {{ 29802790, 16418079 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2010-05-17"), "end-date": null } ] }
+{ "id": 11136910, "id-copy": 11136910, "alias": "Karl", "name": "KarlGarratt", "user-since": datetime("2006-12-22T01:58:50.000Z"), "user-since-copy": datetime("2006-12-22T01:58:50.000Z"), "friend-ids": {{ 753124, 31382435, 30698735, 25951267, 27027532, 34551403, 9451765, 37517863, 3719825, 37613952, 18670991, 39783690, 6592095, 27477830, 31739951, 24458195, 12317249 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2011-05-11"), "end-date": null } ] }
+{ "id": 11139106, "id-copy": 11139106, "alias": "Faith", "name": "FaithHicks", "user-since": datetime("2008-01-08T07:44:36.000Z"), "user-since-copy": datetime("2008-01-08T07:44:36.000Z"), "friend-ids": {{ 5409553, 11995627, 30724106, 17065157, 29513453, 38627025, 34382279, 36487812, 4292416, 19328709, 42169589, 18029462, 20202054, 8738011, 18339448, 2522742, 35366856, 10669527, 44287935, 47124982, 25912125, 38893810, 42212137, 22227146 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2000-11-15"), "end-date": date("2002-10-01") } ] }
+{ "id": 11145823, "id-copy": 11145823, "alias": "Rebeccah", "name": "RebeccahTodd", "user-since": datetime("2007-03-25T15:13:08.000Z"), "user-since-copy": datetime("2007-03-25T15:13:08.000Z"), "friend-ids": {{ 46132741, 11527757, 27573172, 45663865, 45572803, 30569464, 31892238 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2012-04-07"), "end-date": null } ] }
+{ "id": 11175613, "id-copy": 11175613, "alias": "Cuthbert", "name": "CuthbertHoover", "user-since": datetime("2008-04-25T01:12:49.000Z"), "user-since-copy": datetime("2008-04-25T01:12:49.000Z"), "friend-ids": {{ 27333562, 43896730, 6549030, 19576014, 4728367, 15430069, 22146931, 44593208, 14070342, 27801009, 6735368, 35798322, 47213791, 2388166 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2004-07-18"), "end-date": null } ] }
+{ "id": 11196118, "id-copy": 11196118, "alias": "Carson", "name": "CarsonBusk", "user-since": datetime("2006-07-23T07:08:34.000Z"), "user-since-copy": datetime("2006-07-23T07:08:34.000Z"), "friend-ids": {{ 36454884, 31755449, 44569587 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2007-08-13"), "end-date": null } ] }
+{ "id": 11220541, "id-copy": 11220541, "alias": "Phyllida", "name": "PhyllidaRing", "user-since": datetime("2012-03-01T06:11:58.000Z"), "user-since-copy": datetime("2012-03-01T06:11:58.000Z"), "friend-ids": {{ 609357, 45820919, 17439004, 16790980, 27878958, 13930012, 20759108, 23987257, 29330180, 9298668, 10644382, 2596101, 29705735, 13371057, 41709459, 6973880, 41608321, 41344973, 9555209, 37508452, 26445359, 7693361, 12059348 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2002-12-05"), "end-date": date("2009-09-16") } ] }
+{ "id": 11221033, "id-copy": 11221033, "alias": "Vernon", "name": "VernonLear", "user-since": datetime("2006-04-19T13:02:26.000Z"), "user-since-copy": datetime("2006-04-19T13:02:26.000Z"), "friend-ids": {{ 45628776, 31762296, 22963223, 10079920, 20931037, 41768759, 25910794, 41882156, 36691498, 1652094, 25804751, 35757270, 40057670, 37961622, 7430384, 1498630, 7636920, 17109852, 12569850, 47366298, 22902730, 5889994, 21003934, 1929823 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2000-04-18"), "end-date": null } ] }
+{ "id": 11223157, "id-copy": 11223157, "alias": "Lavina", "name": "LavinaPeters", "user-since": datetime("2007-11-08T11:13:48.000Z"), "user-since-copy": datetime("2007-11-08T11:13:48.000Z"), "friend-ids": {{ 45286302 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2012-01-13"), "end-date": null } ] }
+{ "id": 11241523, "id-copy": 11241523, "alias": "Gareth", "name": "GarethFylbrigg", "user-since": datetime("2011-01-05T16:02:25.000Z"), "user-since-copy": datetime("2011-01-05T16:02:25.000Z"), "friend-ids": {{ 45629812, 20113715, 13556523, 29410246, 37849964, 33688575, 35713924, 21492453, 32324177, 5765413, 4491937, 1592640, 2809253, 45152094, 36330032, 25347157, 199553, 16471761, 16621535, 20674800, 42682300, 11354218, 4830164 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2005-10-27"), "end-date": date("2005-12-10") } ] }
+{ "id": 11253043, "id-copy": 11253043, "alias": "Joye", "name": "JoyeGadow", "user-since": datetime("2005-10-03T17:22:30.000Z"), "user-since-copy": datetime("2005-10-03T17:22:30.000Z"), "friend-ids": {{ 24978234, 7896483, 14560795, 18402417, 16619973, 5852675, 29679362, 19344221, 33721635, 14137068, 30581619, 9715250, 10966922, 24167091, 36509340 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2011-01-08"), "end-date": date("2011-08-10") } ] }
+{ "id": 11259028, "id-copy": 11259028, "alias": "Linsay", "name": "LinsayBranson", "user-since": datetime("2011-04-28T08:49:14.000Z"), "user-since-copy": datetime("2011-04-28T08:49:14.000Z"), "friend-ids": {{ 24222662, 814967, 16722114, 24161306, 31611, 2964110, 4912379 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2006-05-18"), "end-date": date("2006-12-16") } ] }
+{ "id": 11273239, "id-copy": 11273239, "alias": "Alanis", "name": "AlanisNeely", "user-since": datetime("2009-04-11T16:49:56.000Z"), "user-since-copy": datetime("2009-04-11T16:49:56.000Z"), "friend-ids": {{ 16788046, 3222185, 46272663, 16782006, 29597609, 9709951, 37694695, 39662749, 18430270, 38598018, 40033174, 34984089, 8435528, 2669100, 18469173, 25201258, 29975180, 16379939, 24603, 2573554, 16344157, 16880724, 2437581 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2001-10-01"), "end-date": date("2006-08-24") } ] }
+{ "id": 11281576, "id-copy": 11281576, "alias": "Louisa", "name": "LouisaWheeler", "user-since": datetime("2005-01-19T05:34:26.000Z"), "user-since-copy": datetime("2005-01-19T05:34:26.000Z"), "friend-ids": {{ 29655724, 29204886, 24086191, 36260050, 502778, 368888, 42853595, 40434954, 46768026, 17096472, 33160972, 15621748, 46246949, 14174435, 99088, 44271646, 3676253, 11744063, 21957250, 34611796, 32735521, 45352911, 6097178, 3796892 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2011-03-05"), "end-date": null } ] }
+{ "id": 11287666, "id-copy": 11287666, "alias": "Darian", "name": "DarianHurst", "user-since": datetime("2009-05-11T03:33:37.000Z"), "user-since-copy": datetime("2009-05-11T03:33:37.000Z"), "friend-ids": {{ 34901893, 38687373, 30369991, 44597588, 41413513, 24197212, 36791517, 19949174, 23092611, 29695794, 7024108, 25202811, 10231736, 3754404, 15863600, 30772236, 21615658 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2012-04-12"), "end-date": date("2012-05-07") } ] }
+{ "id": 11302930, "id-copy": 11302930, "alias": "Eustace", "name": "EustaceKava", "user-since": datetime("2011-08-24T18:08:32.000Z"), "user-since-copy": datetime("2011-08-24T18:08:32.000Z"), "friend-ids": {{ 31173988, 7044500, 11649679, 34385410, 3097267, 24759223, 20452579, 7436501, 4500062, 765860, 14592959, 582267, 25586360, 6035361, 38333776, 47384154, 22158173 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2004-05-24"), "end-date": null } ] }
+{ "id": 11309383, "id-copy": 11309383, "alias": "Lyn", "name": "LynKnapp", "user-since": datetime("2010-07-21T15:29:58.000Z"), "user-since-copy": datetime("2010-07-21T15:29:58.000Z"), "friend-ids": {{ 27610153 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2012-08-28"), "end-date": date("2012-08-29") } ] }
+{ "id": 11321269, "id-copy": 11321269, "alias": "Wilford", "name": "WilfordFuhrer", "user-since": datetime("2012-01-25T14:53:32.000Z"), "user-since-copy": datetime("2012-01-25T14:53:32.000Z"), "friend-ids": {{ 6210425, 27216911, 3113058, 28094966, 119775, 805604, 43386400, 46812881, 22339620, 46498863, 26422270, 43219229, 40022359, 39446155 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2001-07-06"), "end-date": null } ] }
+{ "id": 11364871, "id-copy": 11364871, "alias": "Darrell", "name": "DarrellTaggart", "user-since": datetime("2007-02-14T07:06:21.000Z"), "user-since-copy": datetime("2007-02-14T07:06:21.000Z"), "friend-ids": {{ 42942141, 33727432, 32050372, 39330410, 38031970, 18321427, 4533038, 45054607, 34474798, 29859123, 17215101, 24811589, 12250229, 4712867, 23411515, 10287620, 37707941 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2006-11-26"), "end-date": date("2007-02-18") } ] }
+{ "id": 11380807, "id-copy": 11380807, "alias": "Mckinley", "name": "MckinleyGeyer", "user-since": datetime("2008-02-17T13:01:21.000Z"), "user-since-copy": datetime("2008-02-17T13:01:21.000Z"), "friend-ids": {{ 16655526, 20048717, 15998744, 39702027, 28153175, 40825599, 38372618 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2010-11-26"), "end-date": null } ] }
+{ "id": 11386210, "id-copy": 11386210, "alias": "Dale", "name": "DaleGreenwood", "user-since": datetime("2007-04-17T19:02:45.000Z"), "user-since-copy": datetime("2007-04-17T19:02:45.000Z"), "friend-ids": {{ 3669916 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2002-09-11"), "end-date": null } ] }
+{ "id": 11405905, "id-copy": 11405905, "alias": "Maria", "name": "MariaMoore", "user-since": datetime("2010-05-22T22:23:16.000Z"), "user-since-copy": datetime("2010-05-22T22:23:16.000Z"), "friend-ids": {{ 31883861, 37245457, 28570944, 34781997, 8502652, 44653970, 20757487, 13575261, 13950179, 14347829, 35701908, 35781889, 12226908, 35939258, 5106463, 43910072, 10696743, 21876393, 2309465, 1889615 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2008-03-27"), "end-date": null } ] }
+{ "id": 11412382, "id-copy": 11412382, "alias": "Gosse", "name": "GosseSutton", "user-since": datetime("2011-01-07T02:19:16.000Z"), "user-since-copy": datetime("2011-01-07T02:19:16.000Z"), "friend-ids": {{ 25790586, 42348812, 39275252, 32764855, 11642271, 15982736, 21971689, 13168697, 38246675, 40514837, 20840965 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2010-12-18"), "end-date": date("2011-01-09") } ] }
+{ "id": 11412640, "id-copy": 11412640, "alias": "Larry", "name": "LarryEisaman", "user-since": datetime("2005-04-23T10:38:04.000Z"), "user-since-copy": datetime("2005-04-23T10:38:04.000Z"), "friend-ids": {{ 15063821, 35006785, 18241384, 5967937, 45426140, 44234765, 3244540, 3222784, 36330320 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2001-07-05"), "end-date": null } ] }
+{ "id": 11417455, "id-copy": 11417455, "alias": "Malka", "name": "MalkaWilkinson", "user-since": datetime("2012-04-11T17:22:49.000Z"), "user-since-copy": datetime("2012-04-11T17:22:49.000Z"), "friend-ids": {{ 29261780, 13274200, 41060932, 8851180, 34769837, 3296096, 19488423, 41776348, 44518076, 16669411, 19983817, 26799511, 16166476, 31396373, 4090033, 37968801, 36665813 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2004-03-12"), "end-date": null } ] }
+{ "id": 11417764, "id-copy": 11417764, "alias": "Maren", "name": "MarenDickson", "user-since": datetime("2006-07-20T06:36:52.000Z"), "user-since-copy": datetime("2006-07-20T06:36:52.000Z"), "friend-ids": {{ 14573904, 11946003, 35291176, 25103717, 30010131, 886854, 46625000, 28533752, 46506784, 15300620, 40647607, 10249516, 27751123, 3883546, 41772148, 26655932, 39026036, 4416966, 15070564, 7052224, 10264392, 13650303, 30752174 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2012-08-26"), "end-date": date("2012-08-29") } ] }
+{ "id": 11427025, "id-copy": 11427025, "alias": "Kyran", "name": "KyranKlockman", "user-since": datetime("2007-11-24T11:35:40.000Z"), "user-since-copy": datetime("2007-11-24T11:35:40.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2004-06-10"), "end-date": date("2008-10-25") } ] }
+{ "id": 11427397, "id-copy": 11427397, "alias": "Oscar", "name": "OscarMillhouse", "user-since": datetime("2012-04-07T04:52:39.000Z"), "user-since-copy": datetime("2012-04-07T04:52:39.000Z"), "friend-ids": {{ 27577077, 26831616, 24024317, 24669981, 15864715, 41688094, 25689775, 19288762, 25015698, 24343183, 30170416, 39881555, 29378159, 6748762, 45948007 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2012-05-15"), "end-date": null } ] }
+{ "id": 11441509, "id-copy": 11441509, "alias": "Franklyn", "name": "FranklynZimmer", "user-since": datetime("2012-03-22T13:12:29.000Z"), "user-since-copy": datetime("2012-03-22T13:12:29.000Z"), "friend-ids": {{ 12883110, 5637339, 42139368, 25533619, 19998291, 4231212, 40792266, 9689761, 7591603, 29088602, 40962884, 9432997, 29850101, 47563888, 10384431, 30557751, 9141240, 45176888, 40987369, 42808497, 37891546, 8520042, 12875368, 39706341 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2008-06-09"), "end-date": null } ] }
+{ "id": 11448565, "id-copy": 11448565, "alias": "Martie", "name": "MartiePoley", "user-since": datetime("2010-07-02T14:37:46.000Z"), "user-since-copy": datetime("2010-07-02T14:37:46.000Z"), "friend-ids": {{ 45198632, 14347405, 14595348, 4990646, 44745176, 21949325, 9155582, 3970455, 10097690, 35781298, 46746615, 35535590, 16561713, 31169880, 22467369 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2006-02-08"), "end-date": null } ] }
+{ "id": 11471689, "id-copy": 11471689, "alias": "Bevis", "name": "BevisWhishaw", "user-since": datetime("2011-03-05T23:14:53.000Z"), "user-since-copy": datetime("2011-03-05T23:14:53.000Z"), "friend-ids": {{ 27818002, 43784015, 39101258, 28170566, 38541659, 43935487, 907437, 25457112, 4731176, 35304801, 30364855, 33197014, 27028915, 21746182, 47624076, 41599425, 8592245 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2000-04-04"), "end-date": date("2009-05-08") } ] }
+{ "id": 11494930, "id-copy": 11494930, "alias": "Eleanor", "name": "EleanorAnderson", "user-since": datetime("2008-09-01T04:27:31.000Z"), "user-since-copy": datetime("2008-09-01T04:27:31.000Z"), "friend-ids": {{ 46834294, 32081711 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2008-01-19"), "end-date": null } ] }
+{ "id": 11506045, "id-copy": 11506045, "alias": "Marci", "name": "MarciSaltser", "user-since": datetime("2011-08-05T00:36:14.000Z"), "user-since-copy": datetime("2011-08-05T00:36:14.000Z"), "friend-ids": {{ 44810951, 11599851, 4960763, 13454104, 22872317, 44594135, 15792938 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2002-06-22"), "end-date": date("2009-08-20") } ] }
+{ "id": 11515915, "id-copy": 11515915, "alias": "Hunter", "name": "HunterBash", "user-since": datetime("2011-03-05T16:16:17.000Z"), "user-since-copy": datetime("2011-03-05T16:16:17.000Z"), "friend-ids": {{ 14847122, 46314922, 14414318, 46374290, 45050391, 22617753 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2004-01-20"), "end-date": null } ] }
+{ "id": 11534575, "id-copy": 11534575, "alias": "Sena", "name": "SenaWeidemann", "user-since": datetime("2008-05-25T01:11:53.000Z"), "user-since-copy": datetime("2008-05-25T01:11:53.000Z"), "friend-ids": {{ 8564372, 20258364, 35812476, 36877724, 30983504, 17757915, 42833517 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2003-04-22"), "end-date": null } ] }
+{ "id": 11536078, "id-copy": 11536078, "alias": "Scot", "name": "ScotSwartzbaugh", "user-since": datetime("2007-06-02T13:28:19.000Z"), "user-since-copy": datetime("2007-06-02T13:28:19.000Z"), "friend-ids": {{ 160897, 11035428, 35908585, 14713740, 16036400, 21530456, 31659920, 33439685, 42771513, 42899492, 42315848, 17885118, 12371932, 47219421, 45350312, 33755309, 30284897, 34557464, 21531204, 26093690 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2002-02-23"), "end-date": date("2005-03-24") } ] }
+{ "id": 11542174, "id-copy": 11542174, "alias": "Pacey", "name": "PaceyTripp", "user-since": datetime("2011-11-07T08:36:12.000Z"), "user-since-copy": datetime("2011-11-07T08:36:12.000Z"), "friend-ids": {{ 35602078, 32622628, 34826581, 34837077, 41522736, 14908313, 42986568 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2006-07-08"), "end-date": null } ] }
+{ "id": 11547586, "id-copy": 11547586, "alias": "Rosanne", "name": "RosanneWatkins", "user-since": datetime("2008-03-02T16:07:45.000Z"), "user-since-copy": datetime("2008-03-02T16:07:45.000Z"), "friend-ids": {{ 47389452, 44553302, 30722503, 3892313, 9603884, 12058710, 18459884, 23971280, 39791340, 25400946, 25149383, 8391991, 6548649, 20662585, 34505551, 8352025 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2005-05-17"), "end-date": null } ] }
+{ "id": 11559613, "id-copy": 11559613, "alias": "Mick", "name": "MickWilkinson", "user-since": datetime("2005-12-23T15:11:33.000Z"), "user-since-copy": datetime("2005-12-23T15:11:33.000Z"), "friend-ids": {{ 4641355 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2000-06-03"), "end-date": null } ] }
+{ "id": 11562148, "id-copy": 11562148, "alias": "Rexana", "name": "RexanaStange", "user-since": datetime("2012-08-13T20:11:05.000Z"), "user-since-copy": datetime("2012-08-13T20:11:05.000Z"), "friend-ids": {{ 22418981, 44892347, 43890424, 38530948, 33178064 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2004-11-21"), "end-date": date("2007-11-01") } ] }
+{ "id": 11570386, "id-copy": 11570386, "alias": "Hollis", "name": "HollisIseman", "user-since": datetime("2009-07-11T12:26:25.000Z"), "user-since-copy": datetime("2009-07-11T12:26:25.000Z"), "friend-ids": {{ 28136044, 6945424, 35390131, 12649451, 38331381, 30399822, 47834313 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2011-02-12"), "end-date": null } ] }
+{ "id": 11571085, "id-copy": 11571085, "alias": "Reina", "name": "ReinaWheeler", "user-since": datetime("2010-04-28T08:05:29.000Z"), "user-since-copy": datetime("2010-04-28T08:05:29.000Z"), "friend-ids": {{ 25357083, 40592075, 10585644, 33173927, 42515085 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2000-08-03"), "end-date": null } ] }
+{ "id": 11587057, "id-copy": 11587057, "alias": "Meagan", "name": "MeaganHays", "user-since": datetime("2012-08-15T21:45:05.000Z"), "user-since-copy": datetime("2012-08-15T21:45:05.000Z"), "friend-ids": {{ 26887765, 1940688, 10308941, 42037682, 1716669, 38995955, 17690888, 23227010, 4054166, 22275630, 6863237, 15140164, 38703696, 19044355, 43996569, 12255978, 28516070 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2003-02-26"), "end-date": date("2010-08-05") } ] }
+{ "id": 11591713, "id-copy": 11591713, "alias": "Nannie", "name": "NannieDiller", "user-since": datetime("2008-11-27T08:31:02.000Z"), "user-since-copy": datetime("2008-11-27T08:31:02.000Z"), "friend-ids": {{ 26059738, 32515289, 13702345, 16949001, 10188160, 30251286 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2000-11-27"), "end-date": null } ] }
+{ "id": 11592799, "id-copy": 11592799, "alias": "Booker", "name": "BookerBurkett", "user-since": datetime("2008-07-19T14:13:28.000Z"), "user-since-copy": datetime("2008-07-19T14:13:28.000Z"), "friend-ids": {{ 8693431, 28970363, 8276536, 42506445, 20113337, 40761495 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2010-10-26"), "end-date": date("2010-11-15") } ] }
+{ "id": 11610913, "id-copy": 11610913, "alias": "Vic", "name": "VicDiegel", "user-since": datetime("2008-08-03T21:05:21.000Z"), "user-since-copy": datetime("2008-08-03T21:05:21.000Z"), "friend-ids": {{ 15275871, 8304749, 7803583, 45134147, 36058489, 7180792, 2104280, 4322584, 39304177, 43050196, 32955811, 4161448, 3187410, 47263593 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-03-11"), "end-date": null } ] }
+{ "id": 11616502, "id-copy": 11616502, "alias": "Bernetta", "name": "BernettaMackendoerfer", "user-since": datetime("2005-04-22T03:41:17.000Z"), "user-since-copy": datetime("2005-04-22T03:41:17.000Z"), "friend-ids": {{ 18804036, 29570084, 43932411, 41492349, 46505981, 32524166, 5307968 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2004-08-14"), "end-date": date("2009-08-03") } ] }
+{ "id": 11626564, "id-copy": 11626564, "alias": "Gia", "name": "GiaNehling", "user-since": datetime("2007-05-04T02:40:35.000Z"), "user-since-copy": datetime("2007-05-04T02:40:35.000Z"), "friend-ids": {{ 14435544, 22982758, 14548448, 20359010, 43749230, 6484290, 43513351, 3652065, 1851524, 15523948, 1941233, 47031188, 12649571, 42789428, 10950757, 18325469, 24986924, 39948729, 29738829, 268135, 32952373, 29859037 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2007-06-13"), "end-date": date("2008-07-06") } ] }
+{ "id": 11626678, "id-copy": 11626678, "alias": "Reed", "name": "ReedHaile", "user-since": datetime("2011-05-28T09:52:04.000Z"), "user-since-copy": datetime("2011-05-28T09:52:04.000Z"), "friend-ids": {{ 38955792, 36648350, 7510300, 36168809, 41493759, 45265187, 1653351, 44881482, 44038304 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2012-03-08"), "end-date": date("2012-05-08") } ] }
+{ "id": 11627800, "id-copy": 11627800, "alias": "Andrina", "name": "AndrinaOrbell", "user-since": datetime("2005-01-07T13:18:15.000Z"), "user-since-copy": datetime("2005-01-07T13:18:15.000Z"), "friend-ids": {{ 14378125 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2001-07-27"), "end-date": date("2009-01-26") } ] }
+{ "id": 11693350, "id-copy": 11693350, "alias": "Crystal", "name": "CrystalDickinson", "user-since": datetime("2007-02-08T08:05:12.000Z"), "user-since-copy": datetime("2007-02-08T08:05:12.000Z"), "friend-ids": {{ 32246301, 35277320, 38987334, 3391139, 30437594, 35314588, 32659406, 19055708, 5245289, 1155014, 9266846, 20085529, 27878886, 25128707, 46223557, 16459237, 41315912, 26681594 }}, "employment": [ { "organization-name": "Strongtone", "start-date": date("2011-07-03"), "end-date": date("2011-08-05") } ] }
+{ "id": 11695309, "id-copy": 11695309, "alias": "Petula", "name": "PetulaTanner", "user-since": datetime("2011-12-23T13:29:44.000Z"), "user-since-copy": datetime("2011-12-23T13:29:44.000Z"), "friend-ids": {{ 39411346, 33118908, 44553603 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2003-02-26"), "end-date": date("2007-11-12") } ] }
+{ "id": 11697754, "id-copy": 11697754, "alias": "Jeanette", "name": "JeanetteBullard", "user-since": datetime("2005-11-20T09:56:59.000Z"), "user-since-copy": datetime("2005-11-20T09:56:59.000Z"), "friend-ids": {{ 22439123, 42241829, 21396058, 6050318, 4951741, 4940964, 22719195, 21108984, 1496059, 41986346, 20838301, 34979646, 19524886, 6383593, 37747505, 26787944, 45486736, 7537516 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2006-02-20"), "end-date": null } ] }
+{ "id": 11698384, "id-copy": 11698384, "alias": "Bernetta", "name": "BernettaFiddler", "user-since": datetime("2012-06-20T20:05:46.000Z"), "user-since-copy": datetime("2012-06-20T20:05:46.000Z"), "friend-ids": {{ 12203676 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2000-03-06"), "end-date": null } ] }
+{ "id": 11709478, "id-copy": 11709478, "alias": "Jonty", "name": "JontyCurry", "user-since": datetime("2006-09-08T22:15:05.000Z"), "user-since-copy": datetime("2006-09-08T22:15:05.000Z"), "friend-ids": {{ 1684909, 3914449, 16704128, 11890093, 44073634, 24897496 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2006-03-01"), "end-date": null } ] }
+{ "id": 11713315, "id-copy": 11713315, "alias": "Chung", "name": "ChungStroble", "user-since": datetime("2005-10-20T22:59:27.000Z"), "user-since-copy": datetime("2005-10-20T22:59:27.000Z"), "friend-ids": {{ 13105744, 9160760, 37104436, 33688116, 31455484, 44428287 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2002-12-03"), "end-date": date("2010-10-06") } ] }
+{ "id": 11720794, "id-copy": 11720794, "alias": "Alisha", "name": "AlishaTue", "user-since": datetime("2010-08-11T01:17:31.000Z"), "user-since-copy": datetime("2010-08-11T01:17:31.000Z"), "friend-ids": {{ 6380101, 43972052, 6557931, 42465959, 21268624, 35831867, 45839471, 37781645, 34750475, 35886124, 4491900 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2001-02-02"), "end-date": null } ] }
+{ "id": 11779591, "id-copy": 11779591, "alias": "Galina", "name": "GalinaRoberts", "user-since": datetime("2007-03-18T12:09:38.000Z"), "user-since-copy": datetime("2007-03-18T12:09:38.000Z"), "friend-ids": {{ 16134690, 41543844 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2010-04-17"), "end-date": null } ] }
+{ "id": 11783038, "id-copy": 11783038, "alias": "Cecily", "name": "CecilyRamsey", "user-since": datetime("2011-01-20T23:39:28.000Z"), "user-since-copy": datetime("2011-01-20T23:39:28.000Z"), "friend-ids": {{ 30228589, 45494315, 36823967, 2965036, 37243358, 7140131, 8303981, 10041948, 41439178, 24261471, 16906521, 46190105, 45392996, 21067630, 26632248, 44955893 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2000-03-25"), "end-date": date("2010-06-25") } ] }
+{ "id": 11788096, "id-copy": 11788096, "alias": "Camie", "name": "CamieCressman", "user-since": datetime("2007-10-25T23:38:14.000Z"), "user-since-copy": datetime("2007-10-25T23:38:14.000Z"), "friend-ids": {{ 29310801, 37328820, 47367940, 36796774, 21244245, 7126676, 8254586, 47578674, 39514952, 33623672, 12854915, 6679164, 44128364, 44434013, 20530444, 12243267 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2000-06-20"), "end-date": null } ] }
+{ "id": 11793622, "id-copy": 11793622, "alias": "Leonard", "name": "LeonardAlice", "user-since": datetime("2011-03-02T21:42:07.000Z"), "user-since-copy": datetime("2011-03-02T21:42:07.000Z"), "friend-ids": {{ 38648452, 2302677, 713863, 2484976, 20706899, 6649310, 9952945, 1293945, 23188221, 43521816, 2398744, 28382427, 45933146, 27717079, 12894240, 8077643, 38945982, 12658937, 36047491, 42431984, 43626155 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2001-02-12"), "end-date": date("2001-06-02") } ] }
+{ "id": 11801005, "id-copy": 11801005, "alias": "Jacques", "name": "JacquesWhitling", "user-since": datetime("2007-05-20T05:42:21.000Z"), "user-since-copy": datetime("2007-05-20T05:42:21.000Z"), "friend-ids": {{ 45134681, 48016178 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2006-12-07"), "end-date": null } ] }
+{ "id": 11818252, "id-copy": 11818252, "alias": "Sandee", "name": "SandeeBlair", "user-since": datetime("2008-12-22T20:09:56.000Z"), "user-since-copy": datetime("2008-12-22T20:09:56.000Z"), "friend-ids": {{ 35579096, 13690328, 19410347, 10601941, 13140634, 19728850 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2007-09-24"), "end-date": null } ] }
+{ "id": 11830663, "id-copy": 11830663, "alias": "Bettie", "name": "BettieKing", "user-since": datetime("2009-11-06T15:04:55.000Z"), "user-since-copy": datetime("2009-11-06T15:04:55.000Z"), "friend-ids": {{ 46068058, 35215092, 34850678, 9126970, 16472040, 20000261, 17610567, 37016763, 19830405, 38071058, 43961371, 13092410, 24867008, 12366628, 15539063, 15611017, 1343975, 43254018, 30838755, 30488641, 38027133, 5701592 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2003-04-10"), "end-date": null } ] }
+{ "id": 11867464, "id-copy": 11867464, "alias": "Emmerson", "name": "EmmersonMoore", "user-since": datetime("2006-12-26T00:15:40.000Z"), "user-since-copy": datetime("2006-12-26T00:15:40.000Z"), "friend-ids": {{ 5310233, 16498267, 12436996, 24801626, 44135326, 45729147, 6922158, 25920138, 16324404, 30272475, 22873357, 720070, 9722837, 29718785, 5402637, 287196, 32557949 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2007-06-16"), "end-date": date("2007-02-05") } ] }
+{ "id": 11872177, "id-copy": 11872177, "alias": "Lillie", "name": "LillieLineman", "user-since": datetime("2009-09-28T02:48:03.000Z"), "user-since-copy": datetime("2009-09-28T02:48:03.000Z"), "friend-ids": {{ 16078664, 22307944, 21464886, 40255882, 39090292, 32823112, 5748916, 46831442, 25498280, 268782, 22829744, 17001614 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2006-02-18"), "end-date": null } ] }
+{ "id": 11874358, "id-copy": 11874358, "alias": "Rachyl", "name": "RachylOmara", "user-since": datetime("2008-05-19T19:05:44.000Z"), "user-since-copy": datetime("2008-05-19T19:05:44.000Z"), "friend-ids": {{ 17070163, 39951748, 9940832, 6714785, 4963198, 17121038, 29997771, 21420071, 3672434, 37974288 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2001-11-11"), "end-date": date("2008-07-25") } ] }
+{ "id": 11893462, "id-copy": 11893462, "alias": "Shonna", "name": "ShonnaDickson", "user-since": datetime("2007-06-12T09:36:50.000Z"), "user-since-copy": datetime("2007-06-12T09:36:50.000Z"), "friend-ids": {{ 30462288, 43630666, 35884473, 25217438, 3196051, 41844836, 8922622, 15388786, 33486563, 22739607, 42411271, 47936046, 8921955, 11314832, 13138669, 1057389, 45874085 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2010-06-18"), "end-date": null } ] }
+{ "id": 11899861, "id-copy": 11899861, "alias": "Jacki", "name": "JackiLeach", "user-since": datetime("2009-01-07T13:33:40.000Z"), "user-since-copy": datetime("2009-01-07T13:33:40.000Z"), "friend-ids": {{ 17554995, 17598007, 2855045, 4108843, 47202404, 42565398, 45821410, 32619673, 7988594, 7631349, 20552170, 13116128, 14526615, 17916951, 43018507, 18114607 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2003-06-24"), "end-date": null } ] }
+{ "id": 11920078, "id-copy": 11920078, "alias": "Alane", "name": "AlaneRichter", "user-since": datetime("2005-04-12T04:06:03.000Z"), "user-since-copy": datetime("2005-04-12T04:06:03.000Z"), "friend-ids": {{ 18326190, 34366549, 13047472, 29553920, 6210406, 41865352, 26108964, 15042193, 33225025, 7014329, 11051157, 37032436, 8025322, 21902099, 22953955, 42645725, 29144585 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2006-04-24"), "end-date": null } ] }
+{ "id": 11934781, "id-copy": 11934781, "alias": "Titus", "name": "TitusGertraht", "user-since": datetime("2011-05-02T12:41:28.000Z"), "user-since-copy": datetime("2011-05-02T12:41:28.000Z"), "friend-ids": {{ 32699552, 17016611, 46281182, 32515791, 12860342, 22463323, 33042577, 4477908, 37152051, 5462628, 45666108, 42424199, 44831639, 44546969, 30686685, 40580034 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2000-04-16"), "end-date": null } ] }
+{ "id": 11943412, "id-copy": 11943412, "alias": "Kizzie", "name": "KizzieBillimek", "user-since": datetime("2011-08-25T09:24:43.000Z"), "user-since-copy": datetime("2011-08-25T09:24:43.000Z"), "friend-ids": {{ 47433684, 41380366, 5933545, 6348490, 24429719, 22579519, 21550752, 4653838, 44131628, 7980571, 3208666, 35631166, 13693250, 41263305, 29172668, 24656473, 31110672, 11323134, 23674731, 37422602, 20327470, 13419973 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2012-03-18"), "end-date": date("2012-06-09") } ] }
+{ "id": 11951098, "id-copy": 11951098, "alias": "Tera", "name": "TeraByers", "user-since": datetime("2012-08-03T19:41:26.000Z"), "user-since-copy": datetime("2012-08-03T19:41:26.000Z"), "friend-ids": {{ 15537238, 13699967, 10587728, 23542817, 12703626, 25024772, 19223339, 5547239, 42576945, 27351017, 22726496, 25268071, 4361323, 24631578, 38669047, 44781738, 34646381 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2008-01-04"), "end-date": date("2011-01-14") } ] }
+{ "id": 11951800, "id-copy": 11951800, "alias": "Camron", "name": "CamronBrooks", "user-since": datetime("2006-03-05T19:32:03.000Z"), "user-since-copy": datetime("2006-03-05T19:32:03.000Z"), "friend-ids": {{ 39430755, 45789857, 5352132, 34490450, 39117503, 2233039, 16387184 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2006-12-26"), "end-date": date("2007-11-16") } ] }
+{ "id": 11954992, "id-copy": 11954992, "alias": "Caitlin", "name": "CaitlinLangston", "user-since": datetime("2007-01-02T01:50:34.000Z"), "user-since-copy": datetime("2007-01-02T01:50:34.000Z"), "friend-ids": {{ 23355687, 22474136, 28513847, 32515387, 44041844, 33706721, 10874992, 36341753, 34431157, 16146113, 15462591, 18188151, 29554174, 44940738, 25888018, 42795884, 14382632, 12734889, 11724519, 15830341, 25725320, 37580394, 24124411, 47984339 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2010-05-26"), "end-date": date("2010-03-28") } ] }
+{ "id": 11969527, "id-copy": 11969527, "alias": "Adrian", "name": "AdrianTedrow", "user-since": datetime("2012-02-13T21:27:48.000Z"), "user-since-copy": datetime("2012-02-13T21:27:48.000Z"), "friend-ids": {{ 36940614, 29564878 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2002-01-16"), "end-date": null } ] }
+{ "id": 11981266, "id-copy": 11981266, "alias": "Meghann", "name": "MeghannBatten", "user-since": datetime("2008-06-04T14:25:11.000Z"), "user-since-copy": datetime("2008-06-04T14:25:11.000Z"), "friend-ids": {{ 39206334, 28999157, 22813777 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2012-06-26"), "end-date": null } ] }
+{ "id": 11987626, "id-copy": 11987626, "alias": "Chassidy", "name": "ChassidyHector", "user-since": datetime("2008-07-23T16:16:55.000Z"), "user-since-copy": datetime("2008-07-23T16:16:55.000Z"), "friend-ids": {{ 29831103, 12411598, 20670552, 42569662 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2010-08-22"), "end-date": null } ] }
+{ "id": 9004354, "id-copy": 9004354, "alias": "Deshawn", "name": "DeshawnGarneys", "user-since": datetime("2010-07-21T12:45:03.000Z"), "user-since-copy": datetime("2010-07-21T12:45:03.000Z"), "friend-ids": {{ 46096495, 1526403 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2011-07-08"), "end-date": null } ] }
+{ "id": 9008185, "id-copy": 9008185, "alias": "Francene", "name": "FranceneZoucks", "user-since": datetime("2009-10-18T08:37:00.000Z"), "user-since-copy": datetime("2009-10-18T08:37:00.000Z"), "friend-ids": {{ 47321113, 34578577, 25011033, 19259482, 6221464, 4912987, 20361608, 27957639, 33209653, 46928253, 37111867, 11534180, 31643335, 39967918, 8490889, 23713207, 28827713, 22143989, 21710696, 3545622, 13887489, 41557233, 26554092 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2004-02-01"), "end-date": date("2011-10-10") } ] }
+{ "id": 9020338, "id-copy": 9020338, "alias": "Shenika", "name": "ShenikaColdsmith", "user-since": datetime("2011-02-22T08:03:05.000Z"), "user-since-copy": datetime("2011-02-22T08:03:05.000Z"), "friend-ids": {{ 28029790, 45719398, 12088661, 4134025, 27354070, 46504723, 23155578, 3370020, 26477155, 27314367, 7672726, 41117417 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2011-04-18"), "end-date": null } ] }
+{ "id": 9041992, "id-copy": 9041992, "alias": "Royston", "name": "RoystonBatten", "user-since": datetime("2009-06-27T08:09:45.000Z"), "user-since-copy": datetime("2009-06-27T08:09:45.000Z"), "friend-ids": {{ 35666317, 30439304, 35405688, 2079220, 5996407, 40490306, 33188983, 24455609, 4293738, 29028817, 32566429, 10186823 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2002-04-04"), "end-date": date("2010-06-28") } ] }
+{ "id": 9050866, "id-copy": 9050866, "alias": "Jimmie", "name": "JimmieBicknell", "user-since": datetime("2007-02-15T16:39:19.000Z"), "user-since-copy": datetime("2007-02-15T16:39:19.000Z"), "friend-ids": {{ 17248854, 13830961, 10571254, 637235, 18219702, 4541511, 42876025, 19679892, 14009802, 15312402, 20914286, 41969971, 39807443, 5990836, 1594551, 25853135, 25021671, 21604624, 47574478 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2001-04-09"), "end-date": null } ] }
+{ "id": 9069397, "id-copy": 9069397, "alias": "Manuel", "name": "ManuelTrevithick", "user-since": datetime("2009-01-25T00:11:22.000Z"), "user-since-copy": datetime("2009-01-25T00:11:22.000Z"), "friend-ids": {{ 1121944, 14645273, 16100117, 45331540, 17901062, 7344920, 22533580, 16386626, 4267586, 34975914, 28841442, 38737330, 31607047, 11785331, 9617022, 44328180, 30996836, 14315445, 18464409, 21038654, 14409120, 12230754, 25856707 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2011-10-12"), "end-date": date("2011-03-28") } ] }
+{ "id": 9077020, "id-copy": 9077020, "alias": "Marquis", "name": "MarquisBunten", "user-since": datetime("2008-08-23T04:31:07.000Z"), "user-since-copy": datetime("2008-08-23T04:31:07.000Z"), "friend-ids": {{ 16894897, 21101342, 27872737, 14878739, 47969914, 38986368, 20779589, 4491084, 21066166, 40159242, 25290828, 43152855, 41928030, 2282944 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2001-07-16"), "end-date": null } ] }
+{ "id": 9083791, "id-copy": 9083791, "alias": "Lashay", "name": "LashayLeonard", "user-since": datetime("2008-07-03T04:52:06.000Z"), "user-since-copy": datetime("2008-07-03T04:52:06.000Z"), "friend-ids": {{ 16762687, 32021842, 851915, 36102981, 3553783, 30756474, 12043049, 16852621, 35699568, 4425852, 35227725, 25748188, 9140215, 24886626, 1945167, 12733697, 20761965 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2003-05-19"), "end-date": date("2006-10-16") } ] }
+{ "id": 9102208, "id-copy": 9102208, "alias": "Lottie", "name": "LottieReddish", "user-since": datetime("2007-05-22T00:42:45.000Z"), "user-since-copy": datetime("2007-05-22T00:42:45.000Z"), "friend-ids": {{ 45227463, 22488433, 39033954, 40377121, 17357169, 8890953, 1623690, 11657739, 489001, 26227491, 29459012, 39985553, 3584598, 6381312, 22457740, 43317482, 40035088, 29397671, 18293877, 6788834, 44860241 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2009-04-08"), "end-date": null } ] }
+{ "id": 9129220, "id-copy": 9129220, "alias": "Lessie", "name": "LessieGoodman", "user-since": datetime("2008-09-01T06:07:35.000Z"), "user-since-copy": datetime("2008-09-01T06:07:35.000Z"), "friend-ids": {{ 16418186, 35990435, 22056439, 36479650, 36405609, 12039460, 33551878, 10736746, 41967761, 20046069, 8949956, 26571267 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2004-10-23"), "end-date": date("2011-05-08") } ] }
+{ "id": 9155080, "id-copy": 9155080, "alias": "Errol", "name": "ErrolLittle", "user-since": datetime("2011-12-20T07:09:25.000Z"), "user-since-copy": datetime("2011-12-20T07:09:25.000Z"), "friend-ids": {{ 17400275, 40794627, 12632163, 45365986, 7980045, 7368579, 40357205, 29279590, 258707, 38447445, 27048261, 19911849, 10768265, 24278809, 11940146, 33555290, 23286799, 40641141, 33877442 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2004-03-05"), "end-date": null } ] }
+{ "id": 9158293, "id-copy": 9158293, "alias": "Cortney", "name": "CortneyPainter", "user-since": datetime("2006-03-15T09:03:09.000Z"), "user-since-copy": datetime("2006-03-15T09:03:09.000Z"), "friend-ids": {{ 42832801, 24287760, 37934712, 43376751, 24673433, 14168792, 46862345, 46736573, 21181723, 2094484, 30254710, 45439521, 26589024, 45746175, 13898656, 13470143, 9669892 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2011-06-13"), "end-date": null } ] }
+{ "id": 9168649, "id-copy": 9168649, "alias": "Harmony", "name": "HarmonyMackendoerfer", "user-since": datetime("2006-06-25T21:01:50.000Z"), "user-since-copy": datetime("2006-06-25T21:01:50.000Z"), "friend-ids": {{ 197057, 11973988, 2042364, 21282964, 25761405, 10180346, 39780287, 39243722, 2984620, 7756400, 21311572, 21013939, 16998045, 39135533, 47720897, 20316953 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2005-12-17"), "end-date": date("2009-07-11") } ] }
+{ "id": 9199078, "id-copy": 9199078, "alias": "Erwin", "name": "ErwinErrett", "user-since": datetime("2011-04-20T12:44:31.000Z"), "user-since-copy": datetime("2011-04-20T12:44:31.000Z"), "friend-ids": {{ 31928109, 8101864, 44247743, 21370948 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2000-03-06"), "end-date": null } ] }
+{ "id": 9205615, "id-copy": 9205615, "alias": "Eddie", "name": "EddieRosensteel", "user-since": datetime("2007-01-03T07:17:37.000Z"), "user-since-copy": datetime("2007-01-03T07:17:37.000Z"), "friend-ids": {{ 4208455, 19941893 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2005-11-15"), "end-date": null } ] }
+{ "id": 9259234, "id-copy": 9259234, "alias": "Abigail", "name": "AbigailNicola", "user-since": datetime("2009-08-11T09:18:47.000Z"), "user-since-copy": datetime("2009-08-11T09:18:47.000Z"), "friend-ids": {{ 5465164, 47505082 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2006-02-22"), "end-date": date("2007-10-02") } ] }
+{ "id": 9267007, "id-copy": 9267007, "alias": "Perla", "name": "PerlaCox", "user-since": datetime("2009-04-14T20:56:37.000Z"), "user-since-copy": datetime("2009-04-14T20:56:37.000Z"), "friend-ids": {{ 8937408, 4640163, 41404266, 15668694, 21004833, 12635405, 40379208, 18641131, 14014264, 39008348, 36559306, 26261953, 3593955, 13559713, 34525259 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2003-07-02"), "end-date": null } ] }
+{ "id": 9291964, "id-copy": 9291964, "alias": "Ned", "name": "NedPullman", "user-since": datetime("2011-02-02T07:25:43.000Z"), "user-since-copy": datetime("2011-02-02T07:25:43.000Z"), "friend-ids": {{ 3168566, 3349059, 43400084, 26187570, 11222713, 9924690, 7250860, 9801843, 18856900, 3558502, 17237369, 20047877, 28454433, 12279948, 19319514, 36151797 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2001-08-11"), "end-date": null } ] }
+{ "id": 9295696, "id-copy": 9295696, "alias": "Margaux", "name": "MargauxPerkins", "user-since": datetime("2012-05-23T04:28:13.000Z"), "user-since-copy": datetime("2012-05-23T04:28:13.000Z"), "friend-ids": {{ 23713491, 4271158, 27340057, 7815427, 14232017, 22868851, 2293397, 24147381, 11816307, 16597552, 47120663, 40746124, 9777479, 18134957, 39193317, 19755909, 42252346 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2006-02-17"), "end-date": date("2007-05-06") } ] }
+{ "id": 9297361, "id-copy": 9297361, "alias": "Yasmine", "name": "YasmineBullard", "user-since": datetime("2006-07-11T23:54:23.000Z"), "user-since-copy": datetime("2006-07-11T23:54:23.000Z"), "friend-ids": {{ 27580636, 11448774, 32271178, 9627095, 11487349, 46595708 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2001-10-06"), "end-date": date("2003-03-05") } ] }
+{ "id": 9326218, "id-copy": 9326218, "alias": "Lindsay", "name": "LindsayPaynter", "user-since": datetime("2011-08-27T00:03:13.000Z"), "user-since-copy": datetime("2011-08-27T00:03:13.000Z"), "friend-ids": {{ 3006430, 25941368, 46866627, 21404266, 35141764, 14931901 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-04-06"), "end-date": date("2008-03-02") } ] }
+{ "id": 9341965, "id-copy": 9341965, "alias": "Stephania", "name": "StephaniaBriner", "user-since": datetime("2007-06-15T18:17:32.000Z"), "user-since-copy": datetime("2007-06-15T18:17:32.000Z"), "friend-ids": {{ 9361850, 12128362, 42864061, 6323327, 34867192, 32746507, 17493376, 17276666, 33869929, 20708786 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2004-03-23"), "end-date": date("2009-01-07") } ] }
+{ "id": 9354127, "id-copy": 9354127, "alias": "Seymour", "name": "SeymourFlick", "user-since": datetime("2011-06-17T06:00:11.000Z"), "user-since-copy": datetime("2011-06-17T06:00:11.000Z"), "friend-ids": {{ 7662170, 25563062, 18178019, 32667220, 12254954, 7192061, 18829113, 8959008, 1692176, 28852587, 17130396, 12781461, 4083182, 11054115, 10558861, 13876198 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2007-11-23"), "end-date": null } ] }
+{ "id": 9373819, "id-copy": 9373819, "alias": "Man", "name": "ManHarding", "user-since": datetime("2005-03-19T02:36:47.000Z"), "user-since-copy": datetime("2005-03-19T02:36:47.000Z"), "friend-ids": {{ 10687886, 6212430, 40098775, 8554409, 18917793, 9329327, 38361031, 27404932, 29083756, 28482636, 38832020, 7859160, 14175144, 3316105, 16742847, 8143105, 13049385, 22288103, 36693926, 26571195, 6536981, 32281681, 41798492, 36467563 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2009-02-08"), "end-date": null } ] }
+{ "id": 9379330, "id-copy": 9379330, "alias": "Esther", "name": "EstherReichard", "user-since": datetime("2006-09-23T09:53:43.000Z"), "user-since-copy": datetime("2006-09-23T09:53:43.000Z"), "friend-ids": {{ 29035495, 33601969, 32342695, 28995226, 34638799, 38330225, 38512256 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2006-05-27"), "end-date": null } ] }
+{ "id": 9379975, "id-copy": 9379975, "alias": "Kyra", "name": "KyraLangston", "user-since": datetime("2012-01-18T06:06:56.000Z"), "user-since-copy": datetime("2012-01-18T06:06:56.000Z"), "friend-ids": {{ 46662872, 1388016, 21715152, 3266023, 18080709, 25857347, 29710885, 22300787, 25086634, 25220921, 17189604, 21754574, 27820275, 7441940, 10911235, 46304871, 6518794 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2008-04-03"), "end-date": date("2008-04-07") } ] }
+{ "id": 9395638, "id-copy": 9395638, "alias": "Toby", "name": "TobyThomlinson", "user-since": datetime("2012-02-02T02:11:31.000Z"), "user-since-copy": datetime("2012-02-02T02:11:31.000Z"), "friend-ids": {{ 39086825, 14218540, 37526829, 46631432, 24407673, 19484977, 3657630 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2012-02-26"), "end-date": null } ] }
+{ "id": 9407710, "id-copy": 9407710, "alias": "Todd", "name": "ToddStall", "user-since": datetime("2009-09-21T02:18:16.000Z"), "user-since-copy": datetime("2009-09-21T02:18:16.000Z"), "friend-ids": {{ 46998635, 14217621, 8062100, 47498395, 37234901, 41039045, 37635206, 42173831, 24149948 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2009-09-27"), "end-date": date("2009-07-21") } ] }
+{ "id": 9408427, "id-copy": 9408427, "alias": "Matt", "name": "MattPritchard", "user-since": datetime("2008-10-02T15:31:39.000Z"), "user-since-copy": datetime("2008-10-02T15:31:39.000Z"), "friend-ids": {{ 3596345, 15476624, 33857894, 13004846, 29332890, 23638145, 43402648, 14337754, 3290802, 10537283, 9989868, 33400736, 43952799, 34128983, 3090230, 12591428, 15051691, 7239629, 10295253, 23448932, 30507945 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2012-02-05"), "end-date": null } ] }
+{ "id": 9408688, "id-copy": 9408688, "alias": "Goddard", "name": "GoddardWeisgarber", "user-since": datetime("2011-05-21T13:18:54.000Z"), "user-since-copy": datetime("2011-05-21T13:18:54.000Z"), "friend-ids": {{ 2820008, 31637633, 35026624, 544628, 2552858 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2007-11-26"), "end-date": null } ] }
+{ "id": 9420304, "id-copy": 9420304, "alias": "Alwyn", "name": "AlwynAkers", "user-since": datetime("2009-11-08T08:30:46.000Z"), "user-since-copy": datetime("2009-11-08T08:30:46.000Z"), "friend-ids": {{ 40384671, 13399303, 2163402 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2012-06-14"), "end-date": date("2012-07-17") } ] }
+{ "id": 9421798, "id-copy": 9421798, "alias": "Jaqueline", "name": "JaquelineHasely", "user-since": datetime("2011-06-06T16:32:03.000Z"), "user-since-copy": datetime("2011-06-06T16:32:03.000Z"), "friend-ids": {{ 17911249, 45887650, 15916739, 42045244, 42824039, 4802136, 43709530, 41533233, 13714833, 33000412, 29627102, 43277560, 3727319, 19030370, 47600623, 27902511, 13460397, 34825938, 9726577, 10062858, 34721080, 6725312, 21572679 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2004-02-17"), "end-date": null } ] }
+{ "id": 9446506, "id-copy": 9446506, "alias": "Deshawn", "name": "DeshawnBashline", "user-since": datetime("2009-03-11T18:09:19.000Z"), "user-since-copy": datetime("2009-03-11T18:09:19.000Z"), "friend-ids": {{ 22236205, 44669386, 5098679, 17631352, 40353783, 17155709 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2002-11-12"), "end-date": date("2003-04-22") } ] }
+{ "id": 9449881, "id-copy": 9449881, "alias": "Veola", "name": "VeolaSchaeffer", "user-since": datetime("2005-06-15T04:27:55.000Z"), "user-since-copy": datetime("2005-06-15T04:27:55.000Z"), "friend-ids": {{ 15932585, 16875491, 977001, 15650783, 30629770, 9735829, 35435062, 2023808, 21909452, 29327288, 24004438, 41780113, 10546865, 17514287, 16690971, 23762008, 21853049, 12673064, 35992661, 30579445, 21341455, 2338670 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2001-09-07"), "end-date": null } ] }
+{ "id": 9461098, "id-copy": 9461098, "alias": "Teodoro", "name": "TeodoroBullard", "user-since": datetime("2010-07-24T07:40:44.000Z"), "user-since-copy": datetime("2010-07-24T07:40:44.000Z"), "friend-ids": {{ 8278091, 1756629, 9893864, 11184021, 2292251, 20614604, 48014557, 23491569, 11328678, 11572435, 45790306, 44930978, 34910222, 16655255, 29338869, 27169036, 19669405, 20512510, 33598988, 38104427 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2003-01-17"), "end-date": date("2007-05-28") } ] }
+{ "id": 9467614, "id-copy": 9467614, "alias": "Eloisa", "name": "EloisaEvans", "user-since": datetime("2012-01-20T01:00:51.000Z"), "user-since-copy": datetime("2012-01-20T01:00:51.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2000-11-03"), "end-date": date("2003-01-14") } ] }
+{ "id": 9471385, "id-copy": 9471385, "alias": "Weldon", "name": "WeldonMaclagan", "user-since": datetime("2010-01-24T22:21:59.000Z"), "user-since-copy": datetime("2010-01-24T22:21:59.000Z"), "friend-ids": {{ 42864267, 16710494, 27436346, 7324905, 3901396, 11812437, 31490561, 3906397 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2002-09-07"), "end-date": date("2006-07-08") } ] }
+{ "id": 9480964, "id-copy": 9480964, "alias": "Ava", "name": "AvaCross", "user-since": datetime("2005-11-03T14:59:13.000Z"), "user-since-copy": datetime("2005-11-03T14:59:13.000Z"), "friend-ids": {{ 9693959, 3138151, 20631444, 8672727, 33701530, 14630539, 38539482, 3066915, 30934733, 38630163, 25673376 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2000-06-05"), "end-date": date("2000-10-06") } ] }
+{ "id": 9481756, "id-copy": 9481756, "alias": "Esmaralda", "name": "EsmaraldaAgg", "user-since": datetime("2012-06-26T19:57:38.000Z"), "user-since-copy": datetime("2012-06-26T19:57:38.000Z"), "friend-ids": {{ 40976868 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2008-11-26"), "end-date": date("2008-01-13") } ] }
+{ "id": 9490342, "id-copy": 9490342, "alias": "Gisela", "name": "GiselaTomlinson", "user-since": datetime("2011-10-21T20:36:09.000Z"), "user-since-copy": datetime("2011-10-21T20:36:09.000Z"), "friend-ids": {{ 27609144, 42495049, 21250269, 22561106, 29149509, 16776721, 16980559, 19600765 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2003-02-23"), "end-date": null } ] }
+{ "id": 9505936, "id-copy": 9505936, "alias": "Kerrie", "name": "KerrieGadow", "user-since": datetime("2005-06-26T08:47:14.000Z"), "user-since-copy": datetime("2005-06-26T08:47:14.000Z"), "friend-ids": {{ 46457424, 17421010, 11336465, 19785227 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2004-12-08"), "end-date": date("2010-04-11") } ] }
+{ "id": 9510451, "id-copy": 9510451, "alias": "Chuck", "name": "ChuckFinck", "user-since": datetime("2011-09-10T08:27:31.000Z"), "user-since-copy": datetime("2011-09-10T08:27:31.000Z"), "friend-ids": {{ 5559039, 8997599, 8311284, 20478562, 13734713, 21511695, 30393493 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2001-10-12"), "end-date": null } ] }
+{ "id": 9521401, "id-copy": 9521401, "alias": "Donnette", "name": "DonnetteFaust", "user-since": datetime("2012-03-22T09:38:14.000Z"), "user-since-copy": datetime("2012-03-22T09:38:14.000Z"), "friend-ids": {{ 25050925 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2008-08-20"), "end-date": date("2009-07-09") } ] }
+{ "id": 9532474, "id-copy": 9532474, "alias": "Chester", "name": "ChesterAshmore", "user-since": datetime("2012-02-03T20:36:34.000Z"), "user-since-copy": datetime("2012-02-03T20:36:34.000Z"), "friend-ids": {{ 11340481, 15957237, 47048138, 41603112, 6953329, 6926093, 20866295, 329274, 16187993, 13406075, 34601684, 46151089, 26165473, 2882718, 20731108 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2009-03-14"), "end-date": null } ] }
+{ "id": 9543280, "id-copy": 9543280, "alias": "Isabell", "name": "IsabellGaskins", "user-since": datetime("2009-12-05T01:29:24.000Z"), "user-since-copy": datetime("2009-12-05T01:29:24.000Z"), "friend-ids": {{ 9815607, 43778761, 25835208, 40078303, 28971077, 9802833, 17822058, 12655680, 37398606, 11387722, 5483134, 11506312, 36341116, 13511812, 3504784, 11655484, 18350098, 15365006, 32814750 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2006-07-01"), "end-date": date("2007-08-14") } ] }
+{ "id": 9545626, "id-copy": 9545626, "alias": "Russell", "name": "RussellKeilbach", "user-since": datetime("2010-05-20T15:10:25.000Z"), "user-since-copy": datetime("2010-05-20T15:10:25.000Z"), "friend-ids": {{ 40592323, 28819303 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2003-04-18"), "end-date": null } ] }
+{ "id": 9555157, "id-copy": 9555157, "alias": "Alea", "name": "AleaWallick", "user-since": datetime("2009-11-12T19:32:16.000Z"), "user-since-copy": datetime("2009-11-12T19:32:16.000Z"), "friend-ids": {{ 9936033, 18972695, 22198051, 44425768, 37636218, 25373418, 17204473, 6543589, 23627204, 40204583, 18664982, 27647616, 43332268, 41812682 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2009-02-17"), "end-date": null } ] }
+{ "id": 9562348, "id-copy": 9562348, "alias": "Jefferson", "name": "JeffersonKeister", "user-since": datetime("2005-06-11T01:42:58.000Z"), "user-since-copy": datetime("2005-06-11T01:42:58.000Z"), "friend-ids": {{ 43801762 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2005-07-26"), "end-date": date("2011-12-02") } ] }
+{ "id": 9575338, "id-copy": 9575338, "alias": "Isabell", "name": "IsabellWain", "user-since": datetime("2011-07-05T12:26:43.000Z"), "user-since-copy": datetime("2011-07-05T12:26:43.000Z"), "friend-ids": {{ 42651024, 15652966, 27390748, 19369775, 44130969, 45269514, 210916, 36228917, 31857984, 11676544, 42752689, 14021599, 31749945, 9405328, 37567152, 17083209, 32654328, 39607403, 18699149, 37082017, 6059914, 881724 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2003-06-04"), "end-date": null } ] }
+{ "id": 9577729, "id-copy": 9577729, "alias": "Jann", "name": "JannPorter", "user-since": datetime("2006-05-03T08:57:08.000Z"), "user-since-copy": datetime("2006-05-03T08:57:08.000Z"), "friend-ids": {{ 7711959, 4131696, 10146353, 46418552, 37999454, 38333059, 16381326, 45028736, 16829150 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2006-10-19"), "end-date": null } ] }
+{ "id": 9621157, "id-copy": 9621157, "alias": "Trixie", "name": "TrixieFair", "user-since": datetime("2010-12-25T23:36:49.000Z"), "user-since-copy": datetime("2010-12-25T23:36:49.000Z"), "friend-ids": {{ 17519006, 17545060, 27836293, 11477603, 37895380, 23251592, 12010503, 25406806 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2003-09-23"), "end-date": null } ] }
+{ "id": 9669178, "id-copy": 9669178, "alias": "Gerard", "name": "GerardBeck", "user-since": datetime("2011-04-24T15:49:24.000Z"), "user-since-copy": datetime("2011-04-24T15:49:24.000Z"), "friend-ids": {{ 30087138, 44736614, 1531569 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2003-09-25"), "end-date": date("2005-06-28") } ] }
+{ "id": 9740476, "id-copy": 9740476, "alias": "Tucker", "name": "TuckerRogers", "user-since": datetime("2005-05-22T22:00:09.000Z"), "user-since-copy": datetime("2005-05-22T22:00:09.000Z"), "friend-ids": {{ 13095635, 36113924, 11767777, 15169454, 1692699, 19622409, 17110214 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2009-03-24"), "end-date": date("2011-02-13") } ] }
+{ "id": 9774613, "id-copy": 9774613, "alias": "Kaycee", "name": "KayceeGeyer", "user-since": datetime("2008-12-19T06:09:36.000Z"), "user-since-copy": datetime("2008-12-19T06:09:36.000Z"), "friend-ids": {{ 35485847, 33668074, 21309976, 40428525, 40450508, 30804358, 1365381, 5197688, 37844952, 4076960, 28446817, 20696590, 23896488, 33454126, 21411087, 9300550, 12986775, 36731809, 47850175, 9503217, 22481614, 29556396, 15013896, 14407126 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2001-03-23"), "end-date": date("2003-01-16") } ] }
+{ "id": 9779623, "id-copy": 9779623, "alias": "Alberto", "name": "AlbertoCraig", "user-since": datetime("2009-11-25T14:48:04.000Z"), "user-since-copy": datetime("2009-11-25T14:48:04.000Z"), "friend-ids": {{ 6737836, 26882597, 30254391, 4861442, 18105612 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2012-01-25"), "end-date": null } ] }
+{ "id": 9799264, "id-copy": 9799264, "alias": "Bradley", "name": "BradleyTodd", "user-since": datetime("2011-05-18T23:42:33.000Z"), "user-since-copy": datetime("2011-05-18T23:42:33.000Z"), "friend-ids": {{ 8836368, 35488923, 26777243, 46550104, 9866525, 965209 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2007-12-22"), "end-date": null } ] }
+{ "id": 9799591, "id-copy": 9799591, "alias": "Royston", "name": "RoystonChurchill", "user-since": datetime("2011-01-21T13:57:31.000Z"), "user-since-copy": datetime("2011-01-21T13:57:31.000Z"), "friend-ids": {{ 22757950, 4629721, 19522595, 27737642, 39393176, 9321441, 13496995, 43301849, 3869585, 34993450, 24876688 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2000-07-15"), "end-date": null } ] }
+{ "id": 9802330, "id-copy": 9802330, "alias": "Kirby", "name": "KirbyKnopsnider", "user-since": datetime("2011-12-18T01:10:12.000Z"), "user-since-copy": datetime("2011-12-18T01:10:12.000Z"), "friend-ids": {{ 3703876, 46564552, 9263120, 39930137, 36202804, 45164241, 7778394, 2527495, 2831079, 33834588, 42759211, 2766215, 36344152, 5218620, 1190357, 30615313, 25434877, 43958817, 23617510 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2008-02-01"), "end-date": null } ] }
+{ "id": 9804973, "id-copy": 9804973, "alias": "Harriette", "name": "HarrietteHoopengarner", "user-since": datetime("2011-08-14T20:51:52.000Z"), "user-since-copy": datetime("2011-08-14T20:51:52.000Z"), "friend-ids": {{ 18754696, 27799194, 36904141, 29647419, 8521621, 35146470, 45194388, 43397176, 12596887, 33315, 39826335, 31228413, 123596, 35927645, 11445687, 33208186, 21941268 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2003-03-24"), "end-date": null } ] }
+{ "id": 9812020, "id-copy": 9812020, "alias": "Elias", "name": "EliasBuck", "user-since": datetime("2012-08-03T07:52:34.000Z"), "user-since-copy": datetime("2012-08-03T07:52:34.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2008-07-05"), "end-date": date("2008-12-18") } ] }
+{ "id": 9814867, "id-copy": 9814867, "alias": "Pacey", "name": "PaceyBranson", "user-since": datetime("2011-07-05T06:49:42.000Z"), "user-since-copy": datetime("2011-07-05T06:49:42.000Z"), "friend-ids": {{ 7196953 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2005-11-19"), "end-date": date("2007-12-03") } ] }
+{ "id": 9822973, "id-copy": 9822973, "alias": "Melia", "name": "MeliaWentzel", "user-since": datetime("2012-07-17T05:10:30.000Z"), "user-since-copy": datetime("2012-07-17T05:10:30.000Z"), "friend-ids": {{ 2563633, 27918474, 42233962, 40497985, 4437912, 43013491, 47283180, 20434605, 25309336, 11299381, 20584869, 15093618, 14273412, 46920368, 5868827, 40191100, 44286983, 11787568, 44551406 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2001-07-07"), "end-date": null } ] }
+{ "id": 9878209, "id-copy": 9878209, "alias": "Duana", "name": "DuanaGettemy", "user-since": datetime("2007-03-05T19:06:27.000Z"), "user-since-copy": datetime("2007-03-05T19:06:27.000Z"), "friend-ids": {{ 5530171, 22409344, 22742046, 14418589, 27149252 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2012-08-07"), "end-date": null } ] }
+{ "id": 9886819, "id-copy": 9886819, "alias": "Phoebe", "name": "PhoebeBarnes", "user-since": datetime("2010-12-26T07:30:15.000Z"), "user-since-copy": datetime("2010-12-26T07:30:15.000Z"), "friend-ids": {{ 24361962, 43750816, 46566991, 4790101, 38827567, 6893116, 41555542, 35877264, 18479056, 22186674, 10954414, 43453344, 11903159, 12257863, 45299776 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2000-01-02"), "end-date": date("2008-05-24") } ] }
+{ "id": 9904822, "id-copy": 9904822, "alias": "Judith", "name": "JudithChristman", "user-since": datetime("2005-05-19T14:43:44.000Z"), "user-since-copy": datetime("2005-05-19T14:43:44.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "highfax", "start-date": date("2002-05-06"), "end-date": null } ] }
+{ "id": 9917008, "id-copy": 9917008, "alias": "Clancy", "name": "ClancyHector", "user-since": datetime("2007-09-25T20:55:57.000Z"), "user-since-copy": datetime("2007-09-25T20:55:57.000Z"), "friend-ids": {{ 37754545, 37579706, 39121342, 28434988, 3927416, 3794736, 17107964, 20761621, 20497172, 28562441, 4310488, 35121288, 2380560, 32434056 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2011-06-24"), "end-date": null } ] }
+{ "id": 9945166, "id-copy": 9945166, "alias": "Lilly", "name": "LillyPirl", "user-since": datetime("2009-10-26T11:59:59.000Z"), "user-since-copy": datetime("2009-10-26T11:59:59.000Z"), "friend-ids": {{ 44569094, 5885974, 43165146, 40353390, 45117914, 35995608, 22535699, 46288114, 47171829, 14193764, 45832182, 4957844, 2623547, 37294528 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2009-12-15"), "end-date": date("2011-11-20") } ] }
+{ "id": 9951325, "id-copy": 9951325, "alias": "Sarah", "name": "SarahRockwell", "user-since": datetime("2009-08-25T01:56:51.000Z"), "user-since-copy": datetime("2009-08-25T01:56:51.000Z"), "friend-ids": {{ 14846488, 32939876, 43509116, 36687501, 6496360, 47346160, 20558288, 21828060 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2002-11-18"), "end-date": null } ] }
+{ "id": 9952339, "id-copy": 9952339, "alias": "Dacia", "name": "DaciaStaymates", "user-since": datetime("2009-09-27T09:55:51.000Z"), "user-since-copy": datetime("2009-09-27T09:55:51.000Z"), "friend-ids": {{ 5177020, 46967179, 24156959, 17828131, 41565753, 1929360, 33761670, 27544454, 9964059, 25582191 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2000-10-12"), "end-date": date("2007-01-20") } ] }
+{ "id": 9967888, "id-copy": 9967888, "alias": "Andrea", "name": "AndreaBerry", "user-since": datetime("2007-05-03T20:18:51.000Z"), "user-since-copy": datetime("2007-05-03T20:18:51.000Z"), "friend-ids": {{ 1106859, 38049440, 23056791, 16253206, 7727164, 19267641, 31798723, 30455164, 24738450, 15142413, 15111012, 3782070, 11502933, 44299958, 30277689, 3512757, 41960838, 7667284, 9192069, 12267931, 34901540, 20633036, 37186032, 1734718 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2007-04-01"), "end-date": date("2011-09-07") } ] }
+{ "id": 9970132, "id-copy": 9970132, "alias": "Garrett", "name": "GarrettPery", "user-since": datetime("2007-03-03T11:19:29.000Z"), "user-since-copy": datetime("2007-03-03T11:19:29.000Z"), "friend-ids": {{ 25744707, 31991833, 37406793, 30461766, 24815522, 3640470, 13669903, 17663561, 19222132, 29107132, 42516393, 40032051, 24029037, 28047983, 45579233 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2001-01-12"), "end-date": null } ] }
+{ "id": 9974485, "id-copy": 9974485, "alias": "Leo", "name": "LeoRawls", "user-since": datetime("2005-02-12T12:01:58.000Z"), "user-since-copy": datetime("2005-02-12T12:01:58.000Z"), "friend-ids": {{ 41189338, 33744557, 2485502, 8308490, 43237410 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2002-11-05"), "end-date": date("2009-04-12") } ] }
+{ "id": 9997456, "id-copy": 9997456, "alias": "Micah", "name": "MicahRogers", "user-since": datetime("2008-03-01T05:53:42.000Z"), "user-since-copy": datetime("2008-03-01T05:53:42.000Z"), "friend-ids": {{ 17761154, 33509079, 36866187, 24618619, 7048673, 18747407, 31947241, 33710255, 40699565, 22334622, 24425777, 19450074, 39309621, 4464803, 15881946, 35888289, 10539684, 17175942, 20754578, 27045156, 14301629, 19478576 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2011-02-16"), "end-date": null } ] }
+{ "id": 10000456, "id-copy": 10000456, "alias": "Love", "name": "LoveHawker", "user-since": datetime("2011-03-01T20:42:05.000Z"), "user-since-copy": datetime("2011-03-01T20:42:05.000Z"), "friend-ids": {{ 33646270, 5736885, 35243769, 35528678, 43954964, 44975821, 1839952, 24025196, 1108928 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2010-11-23"), "end-date": date("2011-03-07") } ] }
+{ "id": 10017829, "id-copy": 10017829, "alias": "Adam", "name": "AdamTrovato", "user-since": datetime("2009-04-15T20:21:48.000Z"), "user-since-copy": datetime("2009-04-15T20:21:48.000Z"), "friend-ids": {{ 7572792, 20961281, 47727918, 25262896, 33740076, 14418354, 42807653, 34174665, 12459426, 28777106, 44409513, 39753872, 9172361, 36746114, 196755 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2007-09-25"), "end-date": null } ] }
+{ "id": 10066711, "id-copy": 10066711, "alias": "Nichelle", "name": "NichelleErschoff", "user-since": datetime("2009-11-10T21:17:50.000Z"), "user-since-copy": datetime("2009-11-10T21:17:50.000Z"), "friend-ids": {{ 19024226, 24428716, 24428406, 10686682, 46410623, 45809403, 33158503 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2004-06-21"), "end-date": date("2005-08-01") } ] }
+{ "id": 10073440, "id-copy": 10073440, "alias": "Mat", "name": "MatHasely", "user-since": datetime("2007-02-15T12:28:32.000Z"), "user-since-copy": datetime("2007-02-15T12:28:32.000Z"), "friend-ids": {{ 18317132, 16303558, 35197704, 41199497, 17394418, 18594954, 13332602, 15164806, 20807780, 18284264, 17164369, 6418744, 26535302, 47287046, 7169299, 22825706, 34007482, 38108004, 14449725, 16993574, 28055503 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2005-09-04"), "end-date": date("2006-06-02") } ] }
+{ "id": 10083103, "id-copy": 10083103, "alias": "Albertine", "name": "AlbertineShick", "user-since": datetime("2006-11-10T03:24:02.000Z"), "user-since-copy": datetime("2006-11-10T03:24:02.000Z"), "friend-ids": {{ 22979883, 41779991, 30340160, 44852777, 43786950, 33382165, 898482, 16427018, 1264379, 19925419, 10166319, 12658187, 38802346 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2010-03-05"), "end-date": null } ] }
+{ "id": 10086913, "id-copy": 10086913, "alias": "Margaretta", "name": "MargarettaPfeifer", "user-since": datetime("2012-03-04T14:47:18.000Z"), "user-since-copy": datetime("2012-03-04T14:47:18.000Z"), "friend-ids": {{ 9800482, 3761286, 34428154, 18082184, 14845214, 33053674, 46786785, 22235473, 23677556, 24819784, 47587008, 36939436, 14987278 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2003-07-08"), "end-date": date("2010-03-01") } ] }
+{ "id": 10116496, "id-copy": 10116496, "alias": "Gena", "name": "GenaJerome", "user-since": datetime("2005-03-04T21:38:41.000Z"), "user-since-copy": datetime("2005-03-04T21:38:41.000Z"), "friend-ids": {{ 11698908, 11838778, 10546816, 13504928, 25681727, 20198355, 28316946, 13835662, 16328293, 39540292, 43990464, 31393679, 34806990, 19167324, 8558031, 37794176, 14389975 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2003-10-01"), "end-date": date("2006-06-13") } ] }
+{ "id": 10135477, "id-copy": 10135477, "alias": "Jasmine", "name": "JasmineEva", "user-since": datetime("2009-04-03T11:48:27.000Z"), "user-since-copy": datetime("2009-04-03T11:48:27.000Z"), "friend-ids": {{ 3776073 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2000-11-14"), "end-date": date("2001-05-19") } ] }
+{ "id": 10150873, "id-copy": 10150873, "alias": "Shanice", "name": "ShaniceReiss", "user-since": datetime("2005-07-07T09:46:00.000Z"), "user-since-copy": datetime("2005-07-07T09:46:00.000Z"), "friend-ids": {{ 29208488, 6994033, 13074568, 31547206, 2547580, 15915539, 37448883, 38739687, 33246865, 28231547, 33861348, 44929557, 13977747, 44297013, 22367804 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2002-09-07"), "end-date": date("2006-04-23") } ] }
+{ "id": 10177300, "id-copy": 10177300, "alias": "Chase", "name": "ChaseKnapp", "user-since": datetime("2005-09-27T16:41:30.000Z"), "user-since-copy": datetime("2005-09-27T16:41:30.000Z"), "friend-ids": {{ 12805247, 6093464, 39416190, 35877238, 26583227, 37835412, 46337730, 18107636, 43948720, 21031949, 11688759, 13980476, 25486392, 20775628 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2006-03-07"), "end-date": date("2006-05-09") } ] }
+{ "id": 10178518, "id-copy": 10178518, "alias": "Rudyard", "name": "RudyardMcmullen", "user-since": datetime("2011-05-06T14:57:22.000Z"), "user-since-copy": datetime("2011-05-06T14:57:22.000Z"), "friend-ids": {{ 25647527, 14445589, 47924548, 24945241, 13505530, 39640007, 6132209, 815976, 31529708, 28281922, 17886251, 42402860, 18330827, 13619952 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2006-06-24"), "end-date": null } ] }
+{ "id": 10188805, "id-copy": 10188805, "alias": "Margarita", "name": "MargaritaBrinigh", "user-since": datetime("2011-06-26T06:22:38.000Z"), "user-since-copy": datetime("2011-06-26T06:22:38.000Z"), "friend-ids": {{ 39275311, 42262790, 35041935, 12137373, 8507536 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2004-03-17"), "end-date": null } ] }
+{ "id": 10219465, "id-copy": 10219465, "alias": "Ros", "name": "RosSurrency", "user-since": datetime("2010-04-20T12:07:16.000Z"), "user-since-copy": datetime("2010-04-20T12:07:16.000Z"), "friend-ids": {{ 14365151, 47786936, 41386448, 10958072, 34068903, 28844652, 16749120, 16920092, 7474357, 35730197, 13732713, 26185093, 19486844, 13720196, 7483494, 16709415, 32998666, 31641404, 42939361, 20750447, 44343030, 17559252, 13810932 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2008-12-12"), "end-date": date("2010-05-04") } ] }
+{ "id": 10227844, "id-copy": 10227844, "alias": "Simon", "name": "SimonCoates", "user-since": datetime("2008-09-18T06:23:35.000Z"), "user-since-copy": datetime("2008-09-18T06:23:35.000Z"), "friend-ids": {{ 5847048, 15554997, 1367924, 17223026, 31605674, 38148868, 15521228, 37540102, 4103855, 39184726, 26130198, 43081715, 35929397, 28963043, 10703925 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2011-07-02"), "end-date": null } ] }
+{ "id": 10238749, "id-copy": 10238749, "alias": "Elspeth", "name": "ElspethFilby", "user-since": datetime("2010-02-08T22:55:13.000Z"), "user-since-copy": datetime("2010-02-08T22:55:13.000Z"), "friend-ids": {{ 307224, 16533888 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2011-12-10"), "end-date": null } ] }
+{ "id": 10241767, "id-copy": 10241767, "alias": "Lewin", "name": "LewinBurkett", "user-since": datetime("2008-03-24T21:09:05.000Z"), "user-since-copy": datetime("2008-03-24T21:09:05.000Z"), "friend-ids": {{ 5503, 32598090, 36950887, 22362781, 16089120, 30220805, 6197105, 44773004, 17924848, 36033966, 41338779, 38304288, 18528858, 6384026, 46633327, 18024168, 13983021, 7158391, 31922078, 1082072 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2004-02-17"), "end-date": null } ] }
+{ "id": 10247557, "id-copy": 10247557, "alias": "Shanita", "name": "ShanitaReed", "user-since": datetime("2006-08-01T23:58:30.000Z"), "user-since-copy": datetime("2006-08-01T23:58:30.000Z"), "friend-ids": {{ 39665727, 7906210, 46234266, 15304695, 4362978, 43689749, 11688287, 11377882, 33955818, 29447417, 23667673, 7373357, 45056089, 34964516, 13871603, 41976105, 10661879, 11112019, 17797460 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2004-12-08"), "end-date": date("2005-04-04") } ] }
+{ "id": 10252147, "id-copy": 10252147, "alias": "Concha", "name": "ConchaMckinnon", "user-since": datetime("2009-12-21T03:27:35.000Z"), "user-since-copy": datetime("2009-12-21T03:27:35.000Z"), "friend-ids": {{ 8837048, 7758233, 2108777, 31062874, 34698247, 33766563, 10653492, 25103733, 24629375, 38758275, 37539109, 47252638, 41559516, 41883197, 9608881, 26501553, 39435548, 43307321, 46890131, 29908109 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2011-05-09"), "end-date": null } ] }
+{ "id": 10269349, "id-copy": 10269349, "alias": "Oneida", "name": "OneidaJube", "user-since": datetime("2010-11-18T02:17:28.000Z"), "user-since-copy": datetime("2010-11-18T02:17:28.000Z"), "friend-ids": {{ 12058841, 5816839, 33989309, 42710608, 27128355, 22765769, 30666197, 9009086, 7254731, 41783149, 10080163, 38431373, 35086196, 3607650 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2000-12-10"), "end-date": null } ] }
+{ "id": 10271479, "id-copy": 10271479, "alias": "Leah", "name": "LeahKoepple", "user-since": datetime("2007-10-26T15:57:39.000Z"), "user-since-copy": datetime("2007-10-26T15:57:39.000Z"), "friend-ids": {{ 317362, 43304286, 35630504, 16014770, 43567734, 37946435, 7728583, 45620359, 43235478, 17133820, 22926471, 27438784, 43521614, 235789, 43107565, 21967424, 39119573, 1688079, 5463246, 10081045 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2012-06-14"), "end-date": null } ] }
+{ "id": 10277731, "id-copy": 10277731, "alias": "Gallagher", "name": "GallagherMagor", "user-since": datetime("2007-07-02T07:37:02.000Z"), "user-since-copy": datetime("2007-07-02T07:37:02.000Z"), "friend-ids": {{ 22730683, 9352614, 42748868, 24014877, 21749502, 30751403, 41768964, 13317192, 31877814, 35318552, 26843471, 21232937, 11268529, 21902785 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2006-09-03"), "end-date": null } ] }
+{ "id": 10283941, "id-copy": 10283941, "alias": "Jeffie", "name": "JeffieChappel", "user-since": datetime("2012-06-17T10:07:53.000Z"), "user-since-copy": datetime("2012-06-17T10:07:53.000Z"), "friend-ids": {{ 37665650, 44995551, 8518132, 25975224, 22980129, 41720034, 42152946, 26671472, 25698917, 24270208, 36866555, 6728174, 46967331, 31563323, 1382901, 6764335, 35373496 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2005-01-26"), "end-date": null } ] }
+{ "id": 10284583, "id-copy": 10284583, "alias": "Salal", "name": "SalalButterfill", "user-since": datetime("2011-02-05T13:39:36.000Z"), "user-since-copy": datetime("2011-02-05T13:39:36.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2008-08-10"), "end-date": date("2011-05-02") } ] }
+{ "id": 10301008, "id-copy": 10301008, "alias": "Edgardo", "name": "EdgardoWheeler", "user-since": datetime("2012-04-27T03:11:16.000Z"), "user-since-copy": datetime("2012-04-27T03:11:16.000Z"), "friend-ids": {{ 44525957, 2368018 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2004-07-02"), "end-date": date("2009-04-13") } ] }
+{ "id": 10323868, "id-copy": 10323868, "alias": "Floyd", "name": "FloydCostello", "user-since": datetime("2007-12-17T05:45:55.000Z"), "user-since-copy": datetime("2007-12-17T05:45:55.000Z"), "friend-ids": {{ 16296950, 29360254, 19980961, 43395913, 46984972, 2696536, 9715184, 10851075, 25657111, 46730259, 9182621, 31950695, 46717390, 16664917, 38439464, 6987406, 28167105, 10608129, 11375117, 4306430, 31737185, 29321535, 7420588 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2000-08-21"), "end-date": null } ] }
+{ "id": 10346338, "id-copy": 10346338, "alias": "Caelie", "name": "CaelieYates", "user-since": datetime("2011-11-10T19:17:38.000Z"), "user-since-copy": datetime("2011-11-10T19:17:38.000Z"), "friend-ids": {{ 3910270, 7940512, 32351319, 27966615, 33829964, 34529061, 19420019, 7423616, 22246488, 7284253, 8419860, 43330144 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2005-02-07"), "end-date": date("2011-09-05") } ] }
+{ "id": 10348309, "id-copy": 10348309, "alias": "Bernard", "name": "BernardAltman", "user-since": datetime("2010-09-23T09:08:33.000Z"), "user-since-copy": datetime("2010-09-23T09:08:33.000Z"), "friend-ids": {{ 7859503, 40438517, 7050233, 41735514, 8274833, 12496793, 41853402, 23751827, 23485505, 35520895, 17406459, 20238814, 42333149 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2007-07-27"), "end-date": null } ] }
+{ "id": 10353946, "id-copy": 10353946, "alias": "Cass", "name": "CassPirl", "user-since": datetime("2010-10-25T21:08:28.000Z"), "user-since-copy": datetime("2010-10-25T21:08:28.000Z"), "friend-ids": {{ 43117144, 29185875, 28524977, 4904289, 37353728, 30484159, 40114905, 18108320, 46098949, 30207639 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2006-06-10"), "end-date": null } ] }
+{ "id": 10357477, "id-copy": 10357477, "alias": "Rosy", "name": "RosyMitchell", "user-since": datetime("2005-08-13T13:44:24.000Z"), "user-since-copy": datetime("2005-08-13T13:44:24.000Z"), "friend-ids": {{ 13370964, 4479736, 44060098, 28936173, 42239651, 18380035, 17854869, 36485096, 7662833 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2004-05-12"), "end-date": null } ] }
+{ "id": 10364356, "id-copy": 10364356, "alias": "Katharine", "name": "KatharineHoward", "user-since": datetime("2012-03-04T04:40:32.000Z"), "user-since-copy": datetime("2012-03-04T04:40:32.000Z"), "friend-ids": {{ 38784, 9497194, 38432548, 30160971, 16843331, 36942612, 32507064, 41108421, 31761239, 20202472, 37170299, 39217222, 14201294, 46319310 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2011-11-09"), "end-date": date("2011-07-18") } ] }
+{ "id": 10367416, "id-copy": 10367416, "alias": "Damion", "name": "DamionDean", "user-since": datetime("2008-01-06T05:55:09.000Z"), "user-since-copy": datetime("2008-01-06T05:55:09.000Z"), "friend-ids": {{ 45804001, 13077962, 28346489, 25877214, 10164033, 42903493, 66753, 27961850, 41137249, 20490506 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2007-12-18"), "end-date": null } ] }
+{ "id": 10367503, "id-copy": 10367503, "alias": "Tory", "name": "ToryBender", "user-since": datetime("2012-01-17T03:20:23.000Z"), "user-since-copy": datetime("2012-01-17T03:20:23.000Z"), "friend-ids": {{ 12035968, 32370161, 7506904, 40525754, 44978940, 28927429, 47139832, 9164811, 29534171, 3789973 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2009-02-05"), "end-date": null } ] }
+{ "id": 10384705, "id-copy": 10384705, "alias": "Santos", "name": "SantosJames", "user-since": datetime("2011-05-07T11:54:13.000Z"), "user-since-copy": datetime("2011-05-07T11:54:13.000Z"), "friend-ids": {{ 43937179, 34015979, 7417213, 14660995, 19725400, 3931428, 7318379, 48016396, 44068471, 4577462, 38302695, 16520658, 40487183, 31181305, 11750148, 42688348, 42071075, 10641987, 28860865, 27686448, 40844612, 10817134 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2007-12-18"), "end-date": null } ] }
+{ "id": 10392898, "id-copy": 10392898, "alias": "Rodger", "name": "RodgerLear", "user-since": datetime("2010-03-05T20:39:12.000Z"), "user-since-copy": datetime("2010-03-05T20:39:12.000Z"), "friend-ids": {{ 23638180, 34355575, 28958329, 17287883, 46069191, 4055459, 36969931, 13059600, 6957015, 41374655, 44549230, 1943320, 39878243 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2002-12-22"), "end-date": null } ] }
+{ "id": 10394632, "id-copy": 10394632, "alias": "Marlin", "name": "MarlinLogue", "user-since": datetime("2011-08-28T14:57:40.000Z"), "user-since-copy": datetime("2011-08-28T14:57:40.000Z"), "friend-ids": {{ 45667126 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2004-07-03"), "end-date": date("2009-05-09") } ] }
+{ "id": 10400386, "id-copy": 10400386, "alias": "Marion", "name": "MarionBuck", "user-since": datetime("2006-06-22T03:35:25.000Z"), "user-since-copy": datetime("2006-06-22T03:35:25.000Z"), "friend-ids": {{ 35854700, 8766966, 41860546, 25745457, 12225165, 15412904, 39841282, 5879215, 24965438, 4636142, 43652954, 36414405, 34931848, 38550959, 30395999, 44263220, 8167212, 35555246, 11177002, 29078503 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2000-08-28"), "end-date": null } ] }
+{ "id": 10412287, "id-copy": 10412287, "alias": "Wren", "name": "WrenElizabeth", "user-since": datetime("2009-06-25T07:26:48.000Z"), "user-since-copy": datetime("2009-06-25T07:26:48.000Z"), "friend-ids": {{ 23487913, 35496582, 14824955, 5998721, 10925419, 38937432, 6285652 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2012-07-20"), "end-date": date("2012-07-12") } ] }
+{ "id": 10417531, "id-copy": 10417531, "alias": "Eileen", "name": "EileenCrissman", "user-since": datetime("2009-10-13T21:36:38.000Z"), "user-since-copy": datetime("2009-10-13T21:36:38.000Z"), "friend-ids": {{ 911579, 3590209, 15646563, 31960066, 14495212, 44915460, 42713118, 1962949, 44935091, 6578467, 21896024, 41455809, 25543039, 28884330, 44289305, 15569750, 32580470, 46016098, 9828368 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2003-06-11"), "end-date": date("2005-10-02") } ] }
+{ "id": 10444585, "id-copy": 10444585, "alias": "Harrietta", "name": "HarriettaDunkle", "user-since": datetime("2012-01-26T16:14:19.000Z"), "user-since-copy": datetime("2012-01-26T16:14:19.000Z"), "friend-ids": {{ 9013750, 39577621, 40067238, 24177261, 41169182, 5939218, 13820152, 47741655 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2004-06-13"), "end-date": null } ] }
+{ "id": 10453837, "id-copy": 10453837, "alias": "Leila", "name": "LeilaHunter", "user-since": datetime("2007-12-08T12:41:34.000Z"), "user-since-copy": datetime("2007-12-08T12:41:34.000Z"), "friend-ids": {{ 2310862, 19014920 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2011-02-06"), "end-date": null } ] }
+{ "id": 10473718, "id-copy": 10473718, "alias": "Elissa", "name": "ElissaStainforth", "user-since": datetime("2007-06-20T07:46:54.000Z"), "user-since-copy": datetime("2007-06-20T07:46:54.000Z"), "friend-ids": {{ 1645948, 612724, 46091510, 32750261, 40622752, 10190250, 42030152, 28645649, 27513961 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2003-06-21"), "end-date": date("2011-09-05") } ] }
+{ "id": 10494370, "id-copy": 10494370, "alias": "Maria", "name": "MariaToke", "user-since": datetime("2009-12-06T17:40:38.000Z"), "user-since-copy": datetime("2009-12-06T17:40:38.000Z"), "friend-ids": {{ 28240347, 34042532 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2001-08-08"), "end-date": date("2008-07-09") } ] }
+{ "id": 10541299, "id-copy": 10541299, "alias": "Derrick", "name": "DerrickLarson", "user-since": datetime("2009-09-04T09:42:12.000Z"), "user-since-copy": datetime("2009-09-04T09:42:12.000Z"), "friend-ids": {{ 39544341, 9620318, 40218798, 34927427, 28533075, 44505091, 29066144, 31724565, 46052997, 3011652, 24709291, 24805644, 41125094, 14186985, 24967210, 32420881, 31162758, 2356654, 11854218, 47933360, 9668743, 26801113 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2000-11-25"), "end-date": null } ] }
+{ "id": 10561624, "id-copy": 10561624, "alias": "Marielle", "name": "MarielleBrandenburg", "user-since": datetime("2005-07-17T10:28:02.000Z"), "user-since-copy": datetime("2005-07-17T10:28:02.000Z"), "friend-ids": {{ 1231477, 14598987 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2000-03-06"), "end-date": date("2005-09-25") } ] }
+{ "id": 10582339, "id-copy": 10582339, "alias": "Randall", "name": "RandallDrabble", "user-since": datetime("2006-09-08T10:08:58.000Z"), "user-since-copy": datetime("2006-09-08T10:08:58.000Z"), "friend-ids": {{ 32686522, 24466673, 14026712, 31573032, 14639819, 19975138, 30208386, 24174917, 7234882, 9431452, 18256175, 18934583, 31539286, 46107937, 32747992, 28900739, 40079932, 40674667, 33527888, 45927633, 22350243, 14260823, 19696930, 17970296 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2008-12-13"), "end-date": null } ] }
+{ "id": 10587655, "id-copy": 10587655, "alias": "Del", "name": "DelLester", "user-since": datetime("2006-04-22T06:14:51.000Z"), "user-since-copy": datetime("2006-04-22T06:14:51.000Z"), "friend-ids": {{ 41382268, 41043817, 37053482, 27889226, 5182442, 46241085, 39510378, 25972421, 6234359, 2782513, 27042023, 20476198 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2001-10-18"), "end-date": null } ] }
+{ "id": 10607341, "id-copy": 10607341, "alias": "Evander", "name": "EvanderPycroft", "user-since": datetime("2005-08-09T23:36:46.000Z"), "user-since-copy": datetime("2005-08-09T23:36:46.000Z"), "friend-ids": {{ 46200658, 38004155 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2004-06-13"), "end-date": null } ] }
+{ "id": 10624381, "id-copy": 10624381, "alias": "Ryana", "name": "RyanaKimmons", "user-since": datetime("2007-09-04T15:42:08.000Z"), "user-since-copy": datetime("2007-09-04T15:42:08.000Z"), "friend-ids": {{ 36219003, 5135252, 24653726, 4767631, 21595268, 4154414, 31857818, 9711256, 20793102, 14509650 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2006-06-21"), "end-date": null } ] }
+{ "id": 10661566, "id-copy": 10661566, "alias": "Cathy", "name": "CathyKight", "user-since": datetime("2007-07-17T18:53:31.000Z"), "user-since-copy": datetime("2007-07-17T18:53:31.000Z"), "friend-ids": {{ 19477294, 31919442, 6947933, 16858850, 21921187, 21214480, 19616226, 2133662, 42362248, 7534944, 12953803, 41148200, 30043772, 38130157, 36623612, 45371575, 25019205, 10260656 }}, "employment": [ { "organization-name": "Voltbam", "start-date": date("2008-12-09"), "end-date": date("2008-01-04") } ] }
+{ "id": 10662082, "id-copy": 10662082, "alias": "Colbert", "name": "ColbertFylbrigg", "user-since": datetime("2005-04-09T18:04:54.000Z"), "user-since-copy": datetime("2005-04-09T18:04:54.000Z"), "friend-ids": {{ 25358191, 27442450, 16828484, 16821866, 7010321, 35271072, 32519925, 15521808, 35168957, 36812363, 18888093, 45727757, 30009499, 31505405, 27925036, 47549214, 20290733, 18290760, 36238437, 32377676 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-04-02"), "end-date": null } ] }
+{ "id": 10703185, "id-copy": 10703185, "alias": "Sabina", "name": "SabinaHall", "user-since": datetime("2012-05-18T20:37:33.000Z"), "user-since-copy": datetime("2012-05-18T20:37:33.000Z"), "friend-ids": {{ 432154, 6472603, 35649237, 46598578, 35486135, 44354453 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2002-11-04"), "end-date": date("2011-10-12") } ] }
+{ "id": 10734148, "id-copy": 10734148, "alias": "Allannah", "name": "AllannahHoffhants", "user-since": datetime("2005-11-18T00:54:25.000Z"), "user-since-copy": datetime("2005-11-18T00:54:25.000Z"), "friend-ids": {{ 26897353, 13343289, 1991130, 39024681, 21839148, 38693973, 19132058, 17589948, 13367008, 30389658, 21757614, 45618415, 23559236, 35669455, 22088928, 2531202, 120534, 867017, 8590987, 25956219, 21819960, 41918122, 31042839, 15019901 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2004-10-25"), "end-date": null } ] }
+{ "id": 10738096, "id-copy": 10738096, "alias": "Dori", "name": "DoriAlcocke", "user-since": datetime("2010-05-21T04:59:08.000Z"), "user-since-copy": datetime("2010-05-21T04:59:08.000Z"), "friend-ids": {{ 44039507, 40951102, 39132038, 31982600, 46848423, 43375356, 6188106, 3044041, 38421537, 18640387, 21639042, 11192576, 15659477, 360828, 26875197, 19433881 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2008-02-19"), "end-date": date("2011-03-24") } ] }
+{ "id": 10745974, "id-copy": 10745974, "alias": "Gavin", "name": "GavinWard", "user-since": datetime("2008-11-23T02:59:13.000Z"), "user-since-copy": datetime("2008-11-23T02:59:13.000Z"), "friend-ids": {{ 45290227, 46308273, 4478698, 27613190, 34907694, 36182643 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-01-01"), "end-date": date("2011-01-17") } ] }
+{ "id": 10766221, "id-copy": 10766221, "alias": "Rosalyn", "name": "RosalynBaxter", "user-since": datetime("2009-04-16T15:46:54.000Z"), "user-since-copy": datetime("2009-04-16T15:46:54.000Z"), "friend-ids": {{ 43759575, 1264811, 9906031, 21579594, 45786210, 14876191, 10711745, 25134652, 25426644, 29987806, 1953812, 29568099, 38860088, 7073296, 13746927, 11395655, 36208297, 25317651, 21356968 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2000-07-04"), "end-date": null } ] }
+{ "id": 10767553, "id-copy": 10767553, "alias": "Titty", "name": "TittyCross", "user-since": datetime("2009-02-08T11:38:56.000Z"), "user-since-copy": datetime("2009-02-08T11:38:56.000Z"), "friend-ids": {{ 10869392, 39422025, 23051606, 43241994, 6257807, 37258783, 26946341, 33120713, 6481181, 13410766, 34576024, 42401239, 28793792, 37331232, 5979767 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2000-12-26"), "end-date": date("2006-01-17") } ] }
+{ "id": 10768810, "id-copy": 10768810, "alias": "Gaston", "name": "GastonBender", "user-since": datetime("2008-05-24T17:27:14.000Z"), "user-since-copy": datetime("2008-05-24T17:27:14.000Z"), "friend-ids": {{ 29652235, 40180625, 34608178, 43814186, 9682855, 24692412, 33119254, 20480079, 35147289, 24629496, 1449575 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2010-04-06"), "end-date": null } ] }
+{ "id": 10772929, "id-copy": 10772929, "alias": "Hugh", "name": "HughTrout", "user-since": datetime("2008-01-24T03:16:55.000Z"), "user-since-copy": datetime("2008-01-24T03:16:55.000Z"), "friend-ids": {{ 39704817, 19656412, 37084896, 5219803, 23455492, 14248249, 26973609, 4607440, 25844255, 3032226, 45432192, 47011338, 41460367, 28779211, 31780563, 31808543, 29732190, 1264228, 7989711, 38397890, 7638694, 3002993, 8960147, 46258407 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2010-08-02"), "end-date": date("2010-05-08") } ] }
+{ "id": 10777072, "id-copy": 10777072, "alias": "Fairy", "name": "FairyAgg", "user-since": datetime("2011-08-22T17:08:52.000Z"), "user-since-copy": datetime("2011-08-22T17:08:52.000Z"), "friend-ids": {{ 30447177, 24535470, 1763903, 4456057, 35013322 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2009-02-19"), "end-date": null } ] }
+{ "id": 10786438, "id-copy": 10786438, "alias": "Sherika", "name": "SherikaShick", "user-since": datetime("2005-05-18T21:46:18.000Z"), "user-since-copy": datetime("2005-05-18T21:46:18.000Z"), "friend-ids": {{ 11188876, 12936787, 43459190, 40396919, 7166644, 20299758 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2002-06-09"), "end-date": null } ] }
+{ "id": 10795960, "id-copy": 10795960, "alias": "Hallam", "name": "HallamBousum", "user-since": datetime("2010-04-23T14:02:10.000Z"), "user-since-copy": datetime("2010-04-23T14:02:10.000Z"), "friend-ids": {{ 23447883, 39605256, 41998325 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2007-04-18"), "end-date": null } ] }
+{ "id": 10800157, "id-copy": 10800157, "alias": "Tiara", "name": "TiaraFuhrer", "user-since": datetime("2010-05-24T21:52:36.000Z"), "user-since-copy": datetime("2010-05-24T21:52:36.000Z"), "friend-ids": {{ 34031723 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2003-03-18"), "end-date": date("2005-09-20") } ] }
+{ "id": 10804771, "id-copy": 10804771, "alias": "Delicia", "name": "DeliciaPittman", "user-since": datetime("2008-04-12T01:07:13.000Z"), "user-since-copy": datetime("2008-04-12T01:07:13.000Z"), "friend-ids": {{ 35228090 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2006-08-16"), "end-date": null } ] }
+{ "id": 10808284, "id-copy": 10808284, "alias": "Natalie", "name": "NatalieJewell", "user-since": datetime("2007-04-15T14:17:38.000Z"), "user-since-copy": datetime("2007-04-15T14:17:38.000Z"), "friend-ids": {{ 20839191, 18422391, 2571767, 39525211, 38867255, 13491856 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2005-09-10"), "end-date": date("2011-01-20") } ] }
+{ "id": 10811875, "id-copy": 10811875, "alias": "Giovanni", "name": "GiovanniWarner", "user-since": datetime("2009-05-28T04:20:11.000Z"), "user-since-copy": datetime("2009-05-28T04:20:11.000Z"), "friend-ids": {{ 8005226, 21432611, 4037183, 40486007, 40666777, 24385549, 3686021, 12188144, 33646224, 46365125, 44351069, 34408172, 35904411, 4322876, 18767645, 10007322 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2005-07-18"), "end-date": date("2011-10-24") } ] }
+{ "id": 10827610, "id-copy": 10827610, "alias": "Madelina", "name": "MadelinaCamp", "user-since": datetime("2010-06-08T13:22:59.000Z"), "user-since-copy": datetime("2010-06-08T13:22:59.000Z"), "friend-ids": {{ 35445385, 15924939, 7897517, 15573537, 7234891, 46098859, 877311, 40923818, 45519215, 27332107, 1693386, 21101894, 35225 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2002-10-04"), "end-date": null } ] }
+{ "id": 10833472, "id-copy": 10833472, "alias": "Monica", "name": "MonicaRyals", "user-since": datetime("2009-02-14T18:52:57.000Z"), "user-since-copy": datetime("2009-02-14T18:52:57.000Z"), "friend-ids": {{ 34417058, 24053823, 28067368, 16205470, 24168710, 9064471 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2003-12-03"), "end-date": date("2006-03-07") } ] }
+{ "id": 10835521, "id-copy": 10835521, "alias": "Margeret", "name": "MargeretEve", "user-since": datetime("2010-02-13T16:16:55.000Z"), "user-since-copy": datetime("2010-02-13T16:16:55.000Z"), "friend-ids": {{ 40363275, 44184724, 42855751, 10492711, 561147, 45516609, 38567828, 9695088, 40235757 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2012-06-08"), "end-date": date("2012-06-27") } ] }
+{ "id": 10847359, "id-copy": 10847359, "alias": "Leone", "name": "LeoneWood", "user-since": datetime("2005-07-28T14:24:43.000Z"), "user-since-copy": datetime("2005-07-28T14:24:43.000Z"), "friend-ids": {{ 7650486, 39843416, 43272193, 47152762, 45218041, 45422234, 46812876, 18098636, 47174431, 19091549, 1405281, 46699360, 37961345, 43323551, 46824225, 30700451, 10188790, 16642374, 26570751 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2005-01-22"), "end-date": null } ] }
+{ "id": 10853926, "id-copy": 10853926, "alias": "Kennard", "name": "KennardGarland", "user-since": datetime("2007-11-28T20:40:40.000Z"), "user-since-copy": datetime("2007-11-28T20:40:40.000Z"), "friend-ids": {{ 47687855, 28575858 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2005-07-17"), "end-date": null } ] }
+{ "id": 10856059, "id-copy": 10856059, "alias": "Leland", "name": "LelandMcdonald", "user-since": datetime("2006-09-26T03:35:07.000Z"), "user-since-copy": datetime("2006-09-26T03:35:07.000Z"), "friend-ids": {{ 29735881, 7080599, 14172811, 24274797, 5773081, 2653240, 18151967, 34988676, 6599030, 46463015, 23254278, 37618443, 32396573 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2004-01-17"), "end-date": null } ] }
+{ "id": 10860286, "id-copy": 10860286, "alias": "Albert", "name": "AlbertMills", "user-since": datetime("2005-01-04T04:39:49.000Z"), "user-since-copy": datetime("2005-01-04T04:39:49.000Z"), "friend-ids": {{ 45171802, 36246654, 30029601, 40155304, 4876814, 275363, 46427463, 5698619, 34383185, 47844520, 45026162, 33852471, 36744791, 40565586, 47142152, 42828565 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2012-02-20"), "end-date": date("2012-03-21") } ] }
+{ "id": 10867624, "id-copy": 10867624, "alias": "Fredric", "name": "FredricKimmons", "user-since": datetime("2005-05-14T23:08:00.000Z"), "user-since-copy": datetime("2005-05-14T23:08:00.000Z"), "friend-ids": {{ 25574899, 26822046, 3408550, 40738004, 3813112, 33045116, 9229839, 28557630, 36781441, 23585776 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2010-02-25"), "end-date": date("2011-07-06") } ] }
+{ "id": 10868761, "id-copy": 10868761, "alias": "Peronel", "name": "PeronelGongaware", "user-since": datetime("2010-01-25T14:26:30.000Z"), "user-since-copy": datetime("2010-01-25T14:26:30.000Z"), "friend-ids": {{ 28271989, 41567995, 31926358, 16420360, 15775849, 44023747, 39099521, 4517209, 39890594, 39784644, 43247769, 25427216, 46426794, 37704581, 46477208, 3213706 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2011-12-16"), "end-date": null } ] }
+{ "id": 10878553, "id-copy": 10878553, "alias": "Fido", "name": "FidoWillcox", "user-since": datetime("2007-01-10T01:06:54.000Z"), "user-since-copy": datetime("2007-01-10T01:06:54.000Z"), "friend-ids": {{ 28379360, 45087756, 15173549, 15693878, 23925453, 44178250, 26895550, 35260808, 9946110 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2012-02-09"), "end-date": date("2012-06-24") } ] }
+{ "id": 10878898, "id-copy": 10878898, "alias": "Webster", "name": "WebsterCarr", "user-since": datetime("2006-07-28T21:17:56.000Z"), "user-since-copy": datetime("2006-07-28T21:17:56.000Z"), "friend-ids": {{ 11755002, 37594815, 4340697, 27424145, 22193377, 31509516, 31372689, 47386546, 30347891, 4070454, 18531894, 28306285, 14110568, 17830332 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2002-03-12"), "end-date": null } ] }
+{ "id": 10882393, "id-copy": 10882393, "alias": "Erica", "name": "EricaHynes", "user-since": datetime("2006-09-16T16:39:05.000Z"), "user-since-copy": datetime("2006-09-16T16:39:05.000Z"), "friend-ids": {{ 23491370, 13390922, 19685128, 47763240, 9493285, 10823383, 45076071, 14858340, 12545499, 40367152, 2150593, 45723007, 21362425, 25435409, 776198, 8016739, 21691528, 21036410, 3131225, 20078710, 28405287, 15599245, 39126345, 36208574 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2012-05-14"), "end-date": date("2012-05-22") } ] }
+{ "id": 10883062, "id-copy": 10883062, "alias": "Lamar", "name": "LamarFelbrigge", "user-since": datetime("2005-02-12T03:19:28.000Z"), "user-since-copy": datetime("2005-02-12T03:19:28.000Z"), "friend-ids": {{ 26304238, 21048260, 26614197, 41153844, 17163890, 27772117, 26679939, 22001103, 46907785, 21321841, 46215643, 31285577, 14997749, 46997910, 44367495, 13858871, 20405288, 36784906, 33752927, 30769058, 43188289, 34006518, 23022696 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2012-06-16"), "end-date": null } ] }
+{ "id": 10884241, "id-copy": 10884241, "alias": "Anamaria", "name": "AnamariaMoon", "user-since": datetime("2005-03-28T11:38:17.000Z"), "user-since-copy": datetime("2005-03-28T11:38:17.000Z"), "friend-ids": {{ 21445295, 42154978, 41608378, 3406391, 26013137, 45437958, 22377352, 26150886, 25726611, 31834547, 17506680, 22932063, 16700407, 22939810, 152978, 45307280, 42212660, 30124140, 9494103, 35217706, 41538534, 26586744, 26538590 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2011-09-10"), "end-date": date("2011-02-06") } ] }
+{ "id": 10899544, "id-copy": 10899544, "alias": "Valentine", "name": "ValentineFisher", "user-since": datetime("2008-07-04T14:36:11.000Z"), "user-since-copy": datetime("2008-07-04T14:36:11.000Z"), "friend-ids": {{ 26471524, 781270, 17136010, 12943313, 42125653, 40372131 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2008-09-02"), "end-date": date("2008-01-21") } ] }
+{ "id": 10901332, "id-copy": 10901332, "alias": "Caelie", "name": "CaelieShafer", "user-since": datetime("2011-09-24T05:08:05.000Z"), "user-since-copy": datetime("2011-09-24T05:08:05.000Z"), "friend-ids": {{ 40761096, 31796928, 1066172, 21271172, 41179382, 46260705, 9287042, 37605846, 18083603, 23469027, 45497916, 10102434, 724885, 31794816, 44125905, 46373183, 28321712 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2012-07-04"), "end-date": null } ] }
+{ "id": 10931647, "id-copy": 10931647, "alias": "Bertina", "name": "BertinaStraub", "user-since": datetime("2011-05-25T19:21:43.000Z"), "user-since-copy": datetime("2011-05-25T19:21:43.000Z"), "friend-ids": {{ 12208030, 43810737, 43870253, 20720324, 7601394, 22266404, 21210273, 10076577, 25757258, 1909792, 26189079, 37799329, 24923233, 31687015, 37580896, 44906728, 46928405, 10679805, 14520239, 1690125, 37459202, 36684838, 30982356 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2011-02-17"), "end-date": date("2011-06-20") } ] }
+{ "id": 10936798, "id-copy": 10936798, "alias": "Chang", "name": "ChangBriner", "user-since": datetime("2011-01-21T02:58:13.000Z"), "user-since-copy": datetime("2011-01-21T02:58:13.000Z"), "friend-ids": {{ 44173597, 3293094, 47813131, 8981206, 36324479, 16594808, 20038389, 11223092, 7224123, 10682354, 7270314, 5170866, 10241023, 43090387, 21910381, 36504407, 18319458, 19534667, 14493618, 11394344, 5990164, 35322441 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2004-12-09"), "end-date": date("2006-08-28") } ] }
+{ "id": 10937395, "id-copy": 10937395, "alias": "Madlyn", "name": "MadlynRader", "user-since": datetime("2010-11-11T02:19:12.000Z"), "user-since-copy": datetime("2010-11-11T02:19:12.000Z"), "friend-ids": {{ 8750346, 40237703, 11127018, 23810876, 33862918, 8179642 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2011-03-12"), "end-date": date("2011-12-06") } ] }
+{ "id": 10937893, "id-copy": 10937893, "alias": "Katheleen", "name": "KatheleenEisenmann", "user-since": datetime("2012-06-17T05:15:08.000Z"), "user-since-copy": datetime("2012-06-17T05:15:08.000Z"), "friend-ids": {{ 30129247, 865896, 35091601, 19852276, 43238329, 46057691, 30405091, 3723169, 6577863, 12648596, 34726408, 19178848, 18365491, 28604299, 29242262, 12826786, 19046213, 23320700, 9318080, 35996590, 24812162, 9639554, 33615920, 6507511 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2006-07-26"), "end-date": null } ] }
+{ "id": 10940377, "id-copy": 10940377, "alias": "Lory", "name": "LoryElless", "user-since": datetime("2011-03-21T19:07:17.000Z"), "user-since-copy": datetime("2011-03-21T19:07:17.000Z"), "friend-ids": {{ 38950352, 10596357, 43176277, 27274342, 27082326 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2007-04-19"), "end-date": null } ] }
+{ "id": 10943104, "id-copy": 10943104, "alias": "Prudence", "name": "PrudencePriebe", "user-since": datetime("2006-04-27T21:00:43.000Z"), "user-since-copy": datetime("2006-04-27T21:00:43.000Z"), "friend-ids": {{ 43633941, 38710166, 34456560, 11324015, 21000755, 23356715, 21056830, 27295754 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2012-08-30"), "end-date": null } ] }
+{ "id": 10948003, "id-copy": 10948003, "alias": "August", "name": "AugustHatch", "user-since": datetime("2006-04-11T03:32:56.000Z"), "user-since-copy": datetime("2006-04-11T03:32:56.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2008-12-16"), "end-date": date("2009-01-21") } ] }
+{ "id": 10953628, "id-copy": 10953628, "alias": "Clement", "name": "ClementHoenshell", "user-since": datetime("2009-01-24T03:52:54.000Z"), "user-since-copy": datetime("2009-01-24T03:52:54.000Z"), "friend-ids": {{ 24684431, 16961296, 13566818 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2011-05-07"), "end-date": null } ] }
+{ "id": 10955896, "id-copy": 10955896, "alias": "Felton", "name": "FeltonRiggle", "user-since": datetime("2010-08-18T08:55:19.000Z"), "user-since-copy": datetime("2010-08-18T08:55:19.000Z"), "friend-ids": {{ 9250996, 46302470, 16921353, 21053478, 40274566, 25492381, 7743899 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2008-09-10"), "end-date": date("2009-01-22") } ] }
+{ "id": 10967305, "id-copy": 10967305, "alias": "Harrietta", "name": "HarriettaClewett", "user-since": datetime("2008-05-11T02:34:28.000Z"), "user-since-copy": datetime("2008-05-11T02:34:28.000Z"), "friend-ids": {{ 3346670, 25522849, 46919524, 22773543, 8985252, 43521041, 14951485, 45977993, 21285106, 17023357, 615364, 23079537, 23459313, 31663735, 24201883, 39321873, 47183802, 26870642, 34447310, 4848880, 17078809, 14119447, 39460378 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2012-05-12"), "end-date": date("2012-06-25") } ] }
+{ "id": 10992421, "id-copy": 10992421, "alias": "Ashleigh", "name": "AshleighStroh", "user-since": datetime("2009-10-20T03:03:48.000Z"), "user-since-copy": datetime("2009-10-20T03:03:48.000Z"), "friend-ids": {{ 34581685, 36997971, 29555907, 34868441, 31092587, 9963667, 60170, 19708784, 26201942, 27806479, 40464656, 27628428, 5144660, 44794976, 9937339 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2001-11-04"), "end-date": null } ] }
+{ "id": 11012734, "id-copy": 11012734, "alias": "Jordan", "name": "JordanSadley", "user-since": datetime("2011-02-26T18:40:19.000Z"), "user-since-copy": datetime("2011-02-26T18:40:19.000Z"), "friend-ids": {{ 37319587, 37212468, 3023956, 43125609 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2007-07-03"), "end-date": date("2011-01-25") } ] }
+{ "id": 11016238, "id-copy": 11016238, "alias": "Justy", "name": "JustyShaner", "user-since": datetime("2008-06-17T22:08:29.000Z"), "user-since-copy": datetime("2008-06-17T22:08:29.000Z"), "friend-ids": {{ 23689951, 17071721, 9194411, 34128749, 46316500, 31173605, 32802286, 26107462, 6561314, 9993897, 14746369, 7297148, 41466258 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2003-12-12"), "end-date": date("2007-04-12") } ] }
+{ "id": 11027953, "id-copy": 11027953, "alias": "Angelika", "name": "AngelikaSanner", "user-since": datetime("2010-10-07T04:25:19.000Z"), "user-since-copy": datetime("2010-10-07T04:25:19.000Z"), "friend-ids": {{ 42662440, 6358862, 21758734, 28882210, 28157558, 39027509, 19068795, 45387055, 34737892, 32277859, 44713546, 24617807, 31067294, 12307376, 28568916, 31114183, 13997610, 15405045, 33587810, 32517419, 13452101, 8309328 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2006-02-25"), "end-date": null } ] }
+{ "id": 11064301, "id-copy": 11064301, "alias": "Dave", "name": "DaveNicholas", "user-since": datetime("2007-01-09T09:19:57.000Z"), "user-since-copy": datetime("2007-01-09T09:19:57.000Z"), "friend-ids": {{ 19136340, 40809808, 18774928, 405329, 27436466, 35586548, 16671212, 44582715, 47932437, 22599645, 26281489, 39246487, 39088455, 43696576, 28175190 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2005-02-04"), "end-date": null } ] }
+{ "id": 11066710, "id-copy": 11066710, "alias": "Caryl", "name": "CarylMaugham", "user-since": datetime("2007-02-10T03:38:03.000Z"), "user-since-copy": datetime("2007-02-10T03:38:03.000Z"), "friend-ids": {{ 41776362, 7370825, 35851510, 23733011, 27617379, 39377372, 3043067, 22122576, 11996852, 20708849, 40772627, 20108470, 4141780, 3724555, 31849764, 7347633 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2001-10-15"), "end-date": null } ] }
+{ "id": 11081539, "id-copy": 11081539, "alias": "Haidee", "name": "HaideeStyle", "user-since": datetime("2012-06-13T11:37:34.000Z"), "user-since-copy": datetime("2012-06-13T11:37:34.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2001-03-05"), "end-date": date("2003-11-17") } ] }
+{ "id": 11089501, "id-copy": 11089501, "alias": "Antonette", "name": "AntonetteBrandenburg", "user-since": datetime("2010-01-02T05:42:44.000Z"), "user-since-copy": datetime("2010-01-02T05:42:44.000Z"), "friend-ids": {{ 18054329, 21707156, 1570987, 17610288, 32279976, 10880989, 37459189, 9057880, 46495123, 29331373, 20615029, 22282366, 22218648, 15950453, 30669615, 46097959, 16640911, 15896647 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2004-10-01"), "end-date": date("2009-02-20") } ] }
+{ "id": 11130676, "id-copy": 11130676, "alias": "Krystal", "name": "KrystalDavis", "user-since": datetime("2008-08-18T00:59:11.000Z"), "user-since-copy": datetime("2008-08-18T00:59:11.000Z"), "friend-ids": {{ 44775993, 31503397, 32012007, 16923302, 37099907, 14276165, 40040126, 38310068 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2003-11-21"), "end-date": null } ] }
+{ "id": 11140213, "id-copy": 11140213, "alias": "Montgomery", "name": "MontgomeryWhittier", "user-since": datetime("2007-06-19T17:46:13.000Z"), "user-since-copy": datetime("2007-06-19T17:46:13.000Z"), "friend-ids": {{ 32831460, 6030454, 30437362, 21866470, 17388602, 40815157, 20000967, 47555494, 5818137, 40634742, 21692148, 2365521, 33290069, 46471164, 9192561, 35768343, 7552168, 3577338, 5346012, 31129868 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2008-02-24"), "end-date": null } ] }
+{ "id": 11140483, "id-copy": 11140483, "alias": "Nena", "name": "NenaBullard", "user-since": datetime("2008-02-23T10:24:08.000Z"), "user-since-copy": datetime("2008-02-23T10:24:08.000Z"), "friend-ids": {{ 26438400, 45201681, 12155417, 43414633, 14267296, 40906639, 8768744, 46840439, 43848021, 24521652, 41247005, 44999926, 13062334, 47731182 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2001-05-16"), "end-date": null } ] }
+{ "id": 11158711, "id-copy": 11158711, "alias": "Gwendolen", "name": "GwendolenBousum", "user-since": datetime("2007-07-06T10:35:24.000Z"), "user-since-copy": datetime("2007-07-06T10:35:24.000Z"), "friend-ids": {{ 22558162, 31443428, 22992355, 19452651, 23323540, 41272500, 17328954, 37489389, 35041092, 42476655 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2005-05-23"), "end-date": null } ] }
+{ "id": 11162920, "id-copy": 11162920, "alias": "Michael", "name": "MichaelJohns", "user-since": datetime("2007-12-21T06:52:31.000Z"), "user-since-copy": datetime("2007-12-21T06:52:31.000Z"), "friend-ids": {{ 47587192, 5639113, 24042062, 26141562, 4128346, 25702038, 16421361, 44444678, 30940270, 16928219, 27816662, 37884076, 40854508, 21061894, 42850960, 42453718, 2763269, 16035171, 47650572, 26811622 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2003-02-24"), "end-date": null } ] }
+{ "id": 11162977, "id-copy": 11162977, "alias": "Orson", "name": "OrsonFlick", "user-since": datetime("2010-02-17T21:05:53.000Z"), "user-since-copy": datetime("2010-02-17T21:05:53.000Z"), "friend-ids": {{ 12213318, 19062680, 20035734, 5154338, 24649936, 30379574, 38611249, 36143038, 13393939, 14976281, 34963200, 4510968, 45722224, 18820241 }}, "employment": [ { "organization-name": "Strongtone", "start-date": date("2001-03-14"), "end-date": date("2001-10-15") } ] }
+{ "id": 11174689, "id-copy": 11174689, "alias": "Thao", "name": "ThaoBrandenburg", "user-since": datetime("2012-04-21T05:25:58.000Z"), "user-since-copy": datetime("2012-04-21T05:25:58.000Z"), "friend-ids": {{ 37540210, 3918403, 33043564, 33664166 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2001-08-22"), "end-date": date("2004-11-19") } ] }
+{ "id": 11187373, "id-copy": 11187373, "alias": "Garfield", "name": "GarfieldWible", "user-since": datetime("2009-06-19T05:22:16.000Z"), "user-since-copy": datetime("2009-06-19T05:22:16.000Z"), "friend-ids": {{ 24453777, 20841948, 12224610, 30351943, 17826670, 36119836, 27850423, 4004658, 42610631, 25893845, 46022891, 33018964, 37844844, 1705377, 38811008, 36802000 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2000-02-22"), "end-date": null } ] }
+{ "id": 11188879, "id-copy": 11188879, "alias": "Corrie", "name": "CorrieOsterwise", "user-since": datetime("2011-01-20T21:11:19.000Z"), "user-since-copy": datetime("2011-01-20T21:11:19.000Z"), "friend-ids": {{ 47499393, 41394452, 27330253, 14958477, 14558879, 47694640, 28440147, 3437209, 40720108, 26390443 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2007-08-28"), "end-date": null } ] }
+{ "id": 11190361, "id-copy": 11190361, "alias": "Jancis", "name": "JancisFeufer", "user-since": datetime("2005-08-04T13:00:03.000Z"), "user-since-copy": datetime("2005-08-04T13:00:03.000Z"), "friend-ids": {{ 29421411, 15938833, 13248806, 1321174, 32401361, 34058563, 39735399, 35531531, 2631116, 1167996, 18366452, 45021961, 246133 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2003-08-27"), "end-date": null } ] }
+{ "id": 11224090, "id-copy": 11224090, "alias": "Alayna", "name": "AlaynaHay", "user-since": datetime("2008-12-27T11:44:03.000Z"), "user-since-copy": datetime("2008-12-27T11:44:03.000Z"), "friend-ids": {{ 9220004, 31827642, 27616881, 26175415, 43152043, 36272681, 669731, 40783516, 31718359, 47123044, 24487696, 31178381, 39602057, 2619975, 27562896, 29215321, 35104306, 909466, 18897009, 35295634 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2003-02-01"), "end-date": date("2007-02-07") } ] }
+{ "id": 11244283, "id-copy": 11244283, "alias": "Erica", "name": "EricaTilton", "user-since": datetime("2005-12-10T16:37:41.000Z"), "user-since-copy": datetime("2005-12-10T16:37:41.000Z"), "friend-ids": {{ 9476551, 22631836, 44127713, 32391437, 19413944, 4263930, 17603111, 24077268, 31120069, 30869992, 6040985, 3918705, 17640663, 22515182 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2002-02-05"), "end-date": date("2003-07-03") } ] }
+{ "id": 11246161, "id-copy": 11246161, "alias": "Jemima", "name": "JemimaJube", "user-since": datetime("2009-10-13T13:44:48.000Z"), "user-since-copy": datetime("2009-10-13T13:44:48.000Z"), "friend-ids": {{ 35264732, 26686176, 37947249, 9511009, 20544975, 21318354, 2417039, 15051823, 23702057, 34446389, 15435804, 42646090, 14791709 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2012-02-26"), "end-date": null } ] }
+{ "id": 11268778, "id-copy": 11268778, "alias": "Chuck", "name": "ChuckRamos", "user-since": datetime("2005-09-24T12:19:57.000Z"), "user-since-copy": datetime("2005-09-24T12:19:57.000Z"), "friend-ids": {{ 2142650, 15399676, 40659179, 32507535, 32269323, 46947373, 46293990, 4237301, 41447393, 21345670, 47299716, 8515646, 27204593, 6676856, 21757183, 13647535, 28951520, 23198255, 1618106, 18189425, 46835891, 7056692, 26622607 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2004-06-24"), "end-date": date("2006-01-05") } ] }
+{ "id": 11269867, "id-copy": 11269867, "alias": "Bettye", "name": "BettyeTeagarden", "user-since": datetime("2006-02-15T08:28:04.000Z"), "user-since-copy": datetime("2006-02-15T08:28:04.000Z"), "friend-ids": {{ 3227122, 9086278, 26175058, 16380287, 15179776, 6343969, 15198730, 7420831, 38504400, 5337815, 35914644, 42885098, 2521174, 43359140, 17884442, 3131060, 35723204, 14956242, 78003, 7455524, 3371831, 46465463, 9947087 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2000-07-21"), "end-date": date("2007-10-28") } ] }
+{ "id": 11270020, "id-copy": 11270020, "alias": "Ursula", "name": "UrsulaSauter", "user-since": datetime("2006-09-17T06:18:31.000Z"), "user-since-copy": datetime("2006-09-17T06:18:31.000Z"), "friend-ids": {{ 13370394, 5537385, 6651824, 27208272, 3304500, 26518061, 44906267, 27803333, 8618582, 22074752, 20865682, 15343007 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2006-08-01"), "end-date": null } ] }
+{ "id": 11276305, "id-copy": 11276305, "alias": "Salome", "name": "SalomeGongaware", "user-since": datetime("2007-06-05T10:15:14.000Z"), "user-since-copy": datetime("2007-06-05T10:15:14.000Z"), "friend-ids": {{ 17354378, 35576200, 42905756, 44408264, 45572153, 18424890, 39234162, 42837501, 38464194, 45237502, 30396078, 16316605, 32231800, 35417394, 32796520, 13885091, 31520983, 4624403, 18144193, 45707906, 8211336, 2864876 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2002-03-16"), "end-date": null } ] }
+{ "id": 11287327, "id-copy": 11287327, "alias": "Vito", "name": "VitoMoffat", "user-since": datetime("2008-02-08T03:16:42.000Z"), "user-since-copy": datetime("2008-02-08T03:16:42.000Z"), "friend-ids": {{ 36850894, 16346016, 4072987, 36112362, 13277841, 24976604, 20216096, 36253616, 13624540, 39256929, 8411929, 13545093, 27563972, 4306316, 9819682, 21998450, 16647991, 1987261 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2001-07-08"), "end-date": date("2005-04-23") } ] }
+{ "id": 11289733, "id-copy": 11289733, "alias": "Jettie", "name": "JettieElinor", "user-since": datetime("2006-03-02T09:44:17.000Z"), "user-since-copy": datetime("2006-03-02T09:44:17.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2002-07-25"), "end-date": date("2005-01-16") } ] }
+{ "id": 11318329, "id-copy": 11318329, "alias": "April", "name": "AprilSurrency", "user-since": datetime("2008-09-02T21:07:03.000Z"), "user-since-copy": datetime("2008-09-02T21:07:03.000Z"), "friend-ids": {{ 8646916, 27873471, 41336682, 42549624, 39851926, 29548550, 31209458, 40169445, 27695329, 20395537, 10311481, 47078664, 32368262, 6850643, 26890752 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2009-12-11"), "end-date": null } ] }
+{ "id": 11327029, "id-copy": 11327029, "alias": "Mallory", "name": "MalloryHughes", "user-since": datetime("2007-08-06T22:11:46.000Z"), "user-since-copy": datetime("2007-08-06T22:11:46.000Z"), "friend-ids": {{ 38924183, 22042572, 21014848, 46309217, 1120998, 19755064, 4413438, 38855205, 17626985, 5727472, 1293238 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2006-02-28"), "end-date": date("2006-08-24") } ] }
+{ "id": 11327731, "id-copy": 11327731, "alias": "Duncan", "name": "DuncanPennington", "user-since": datetime("2007-09-08T05:38:28.000Z"), "user-since-copy": datetime("2007-09-08T05:38:28.000Z"), "friend-ids": {{ 7591038, 8046115, 16606742, 39494564, 32760725, 39036737, 9937167, 38968828, 32536611 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2003-12-06"), "end-date": null } ] }
+{ "id": 11341747, "id-copy": 11341747, "alias": "Margaux", "name": "MargauxBynum", "user-since": datetime("2009-01-16T19:54:27.000Z"), "user-since-copy": datetime("2009-01-16T19:54:27.000Z"), "friend-ids": {{ 27056110, 1770280, 17190314, 18164827, 32684926, 32410281, 27173037, 16864868, 4664026, 31170366, 4296651 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2008-08-20"), "end-date": null } ] }
+{ "id": 11347261, "id-copy": 11347261, "alias": "Linda", "name": "LindaBaldwin", "user-since": datetime("2010-04-21T08:05:44.000Z"), "user-since-copy": datetime("2010-04-21T08:05:44.000Z"), "friend-ids": {{ 1423464, 7534626, 19522889, 25132532, 19933077, 36713596, 31725151, 46644015, 17758352, 37356325, 43714985, 29437022, 21616894, 32487769, 18527683, 32632034, 5598064, 47187635, 23490346 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2005-06-22"), "end-date": date("2007-02-18") } ] }
+{ "id": 11348449, "id-copy": 11348449, "alias": "Domitila", "name": "DomitilaPolson", "user-since": datetime("2009-09-24T21:31:17.000Z"), "user-since-copy": datetime("2009-09-24T21:31:17.000Z"), "friend-ids": {{ 46755392, 24913792, 47792230, 2451253, 10548653, 3083052, 20700516, 15133622, 17284439, 40871072, 6444103, 44749243, 45289097, 19631062, 8873017, 6262067, 4742977, 672148, 19303779 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2005-06-03"), "end-date": null } ] }
+{ "id": 11355979, "id-copy": 11355979, "alias": "Sal", "name": "SalChapman", "user-since": datetime("2012-07-23T17:03:04.000Z"), "user-since-copy": datetime("2012-07-23T17:03:04.000Z"), "friend-ids": {{ 4959799, 33919735, 33624568, 9885012, 16788595, 39510500, 34856818, 22167281, 44317359, 45181449, 43901851, 42402339, 9573000, 16655168 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2006-12-10"), "end-date": null } ] }
+{ "id": 11366056, "id-copy": 11366056, "alias": "Devin", "name": "DevinUlery", "user-since": datetime("2011-05-03T13:27:51.000Z"), "user-since-copy": datetime("2011-05-03T13:27:51.000Z"), "friend-ids": {{ 25443767, 42385070, 31515075, 31340661, 25371541, 34378389, 40381786, 23698797, 40141450, 12814851, 41414503, 39733660, 27910438, 44106204, 18806338, 37909692, 12502759, 4270087, 5110443, 14347603, 19313129, 8826229 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2001-12-15"), "end-date": null } ] }
+{ "id": 11476339, "id-copy": 11476339, "alias": "Hopkin", "name": "HopkinNicholas", "user-since": datetime("2008-09-23T20:48:07.000Z"), "user-since-copy": datetime("2008-09-23T20:48:07.000Z"), "friend-ids": {{ 30021024, 29046949, 8412580, 10700657, 15739611, 36768609 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2004-01-02"), "end-date": null } ] }
+{ "id": 11488420, "id-copy": 11488420, "alias": "Rik", "name": "RikSell", "user-since": datetime("2011-04-24T10:10:24.000Z"), "user-since-copy": datetime("2011-04-24T10:10:24.000Z"), "friend-ids": {{ 37808691, 28841986, 27850488, 28093210, 9165013, 45941806, 5194022, 39773028, 45473967, 44833113, 27429268 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2002-09-23"), "end-date": date("2010-06-23") } ] }
+{ "id": 11515477, "id-copy": 11515477, "alias": "Kassandra", "name": "KassandraByers", "user-since": datetime("2005-05-24T10:27:06.000Z"), "user-since-copy": datetime("2005-05-24T10:27:06.000Z"), "friend-ids": {{ 23979652, 25789717, 7769765, 30747470, 30667193, 22447318, 42934938, 24601934, 31839813, 18960206, 30913033, 39059809, 18213877, 3731518, 10573130, 37902022 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2004-01-13"), "end-date": null } ] }
+{ "id": 11515828, "id-copy": 11515828, "alias": "Christa", "name": "ChristaWain", "user-since": datetime("2007-05-01T13:32:18.000Z"), "user-since-copy": datetime("2007-05-01T13:32:18.000Z"), "friend-ids": {{ 9081871, 27897837, 47641133, 1224070, 41007475, 39553691, 10757036, 28663201, 44842180, 24894191, 42128523, 30703082, 27281648, 9786943 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2012-05-04"), "end-date": null } ] }
+{ "id": 11525302, "id-copy": 11525302, "alias": "Marissa", "name": "MarissaEndsley", "user-since": datetime("2006-09-26T08:55:36.000Z"), "user-since-copy": datetime("2006-09-26T08:55:36.000Z"), "friend-ids": {{ 35476434, 12502442, 19198691, 35401830, 14414490, 11372357, 28886265, 3490052, 13587860, 8127851, 20732439, 44816539, 6616740, 12785784, 16907259, 10942007, 26207, 21026660, 39284170, 25761798, 20688453, 45805952, 15912564 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2007-10-07"), "end-date": date("2010-09-09") } ] }
+{ "id": 11525575, "id-copy": 11525575, "alias": "Zack", "name": "ZackMills", "user-since": datetime("2007-10-15T20:53:30.000Z"), "user-since-copy": datetime("2007-10-15T20:53:30.000Z"), "friend-ids": {{ 11119738, 47490530, 18951399, 24413247, 4019030, 39064308, 43279140, 11316225, 15383674, 40613636, 4793869, 21591307, 23561981, 3763992, 32892218, 34334911, 40693733 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2012-05-25"), "end-date": date("2012-07-09") } ] }
+{ "id": 11533327, "id-copy": 11533327, "alias": "Miguel", "name": "MiguelSteiner", "user-since": datetime("2007-12-08T18:21:30.000Z"), "user-since-copy": datetime("2007-12-08T18:21:30.000Z"), "friend-ids": {{ 41619494, 4881397, 29302201, 26654760, 9690024, 15599321, 37163728, 2420315, 46258007, 15076674, 6757461 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2001-08-19"), "end-date": date("2008-10-15") } ] }
+{ "id": 11540278, "id-copy": 11540278, "alias": "Flora", "name": "FloraSaltser", "user-since": datetime("2007-11-20T08:52:26.000Z"), "user-since-copy": datetime("2007-11-20T08:52:26.000Z"), "friend-ids": {{ 44172124, 43836609, 2821020, 356092, 25456578, 14806637, 19970466, 15369859, 23267393, 34480680, 42574031, 39606777, 17221367, 19617483, 1364901, 21402012, 4999365, 31098654, 34512618, 44652673, 14757091, 9755310, 39190510 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2012-07-07"), "end-date": null } ] }
+{ "id": 11542519, "id-copy": 11542519, "alias": "Colten", "name": "ColtenDemuth", "user-since": datetime("2012-02-09T01:22:04.000Z"), "user-since-copy": datetime("2012-02-09T01:22:04.000Z"), "friend-ids": {{ 15666280, 36489446, 45424145, 47509110, 24198688, 42545568, 30526545, 43828073, 26402530, 23632737, 20385217, 35055795, 38789042, 34967858, 521531, 47834820, 20307524 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2008-04-10"), "end-date": null } ] }
+{ "id": 11570326, "id-copy": 11570326, "alias": "Linden", "name": "LindenFilby", "user-since": datetime("2007-08-16T03:11:11.000Z"), "user-since-copy": datetime("2007-08-16T03:11:11.000Z"), "friend-ids": {{ 6549689, 15243636, 3147666 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2010-02-23"), "end-date": date("2010-04-22") } ] }
+{ "id": 11571217, "id-copy": 11571217, "alias": "Modesto", "name": "ModestoPark", "user-since": datetime("2006-01-18T06:28:01.000Z"), "user-since-copy": datetime("2006-01-18T06:28:01.000Z"), "friend-ids": {{ 3765450, 13287809, 17696557, 32161653, 46823306, 2818286, 38794110, 24894266, 33129431, 26474332, 9356762, 38679272, 40502952, 34470547, 30005230, 32074010, 38611550 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2001-09-01"), "end-date": date("2003-04-11") } ] }
+{ "id": 11598403, "id-copy": 11598403, "alias": "Jo", "name": "JoCattley", "user-since": datetime("2008-01-04T03:33:03.000Z"), "user-since-copy": datetime("2008-01-04T03:33:03.000Z"), "friend-ids": {{ 28948698, 9851844, 31708351, 28418023, 33052184, 24995451, 2840550, 19426008, 3790086 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2006-09-15"), "end-date": null } ] }
+{ "id": 11619817, "id-copy": 11619817, "alias": "Conor", "name": "ConorIsaman", "user-since": datetime("2007-07-19T03:08:58.000Z"), "user-since-copy": datetime("2007-07-19T03:08:58.000Z"), "friend-ids": {{ 3118516, 11993690, 44936801, 20826732, 45978958, 5214526, 29651996, 39212065, 47935248, 13306157, 33084407, 537249, 42089040, 7553609, 42024531, 23482433, 45497814, 26865252, 42135224, 41353574, 28567135, 7898064 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2002-04-26"), "end-date": null } ] }
+{ "id": 11626156, "id-copy": 11626156, "alias": "Laurine", "name": "LaurineBastion", "user-since": datetime("2012-05-14T21:34:43.000Z"), "user-since-copy": datetime("2012-05-14T21:34:43.000Z"), "friend-ids": {{ 13978691, 24432513, 41105156, 4981880 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2000-03-09"), "end-date": null } ] }
+{ "id": 11630158, "id-copy": 11630158, "alias": "Jewel", "name": "JewelPrechtl", "user-since": datetime("2008-09-24T10:05:42.000Z"), "user-since-copy": datetime("2008-09-24T10:05:42.000Z"), "friend-ids": {{ 17110258, 26859370, 7070027, 19698792, 10087924, 31999744, 35694569, 10315290, 15006946, 25258889, 8036893, 20721778, 31250890, 31525573 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2002-10-09"), "end-date": null } ] }
+{ "id": 11633284, "id-copy": 11633284, "alias": "Quinn", "name": "QuinnMillhouse", "user-since": datetime("2006-08-06T07:42:49.000Z"), "user-since-copy": datetime("2006-08-06T07:42:49.000Z"), "friend-ids": {{ 15791690, 46827169, 41678324, 25101779, 24496106, 29442447, 29240215, 23819212, 11076551, 27248100, 1506119, 37415860 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2008-01-06"), "end-date": null } ] }
+{ "id": 11633326, "id-copy": 11633326, "alias": "Jodi", "name": "JodiBrindle", "user-since": datetime("2009-01-02T19:57:58.000Z"), "user-since-copy": datetime("2009-01-02T19:57:58.000Z"), "friend-ids": {{ 5287281, 24414393, 31942570, 45025515, 35679462, 45244705, 4931287, 11590610, 39846242, 14999029, 38735562, 6275771, 33435194 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2009-11-25"), "end-date": null } ] }
+{ "id": 11642026, "id-copy": 11642026, "alias": "Brenden", "name": "BrendenLucy", "user-since": datetime("2010-09-18T13:14:17.000Z"), "user-since-copy": datetime("2010-09-18T13:14:17.000Z"), "friend-ids": {{ 4037044, 13420154, 10023579, 7611523, 10090302, 36514218, 24369151, 10481696, 341494 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2007-07-05"), "end-date": null } ] }
+{ "id": 11659237, "id-copy": 11659237, "alias": "Orlando", "name": "OrlandoMcloskey", "user-since": datetime("2006-09-15T00:02:58.000Z"), "user-since-copy": datetime("2006-09-15T00:02:58.000Z"), "friend-ids": {{ 18927260, 17411696, 20569511, 5242025, 18974872, 24923117, 42416784, 37339853, 42886763, 12241986, 40609114, 8814896, 30383771, 23631329, 41937811, 13354366, 40113344, 11968348, 23416173, 1546554, 46467044, 5542363, 32084191, 3049632 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2006-04-20"), "end-date": null } ] }
+{ "id": 11670739, "id-copy": 11670739, "alias": "Rudyard", "name": "RudyardErrett", "user-since": datetime("2005-03-08T18:26:12.000Z"), "user-since-copy": datetime("2005-03-08T18:26:12.000Z"), "friend-ids": {{ 13253132, 38903405, 45479471, 11551894, 44803858, 34016119, 2477206, 27909363, 2584557, 29078732, 13687500, 1038800, 14467502, 3369722, 11731177, 15702876, 37034289, 21943459 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2011-01-20"), "end-date": null } ] }
+{ "id": 11675221, "id-copy": 11675221, "alias": "Calanthe", "name": "CalantheGearhart", "user-since": datetime("2007-06-08T02:44:20.000Z"), "user-since-copy": datetime("2007-06-08T02:44:20.000Z"), "friend-ids": {{ 19185575 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2010-05-21"), "end-date": null } ] }
+{ "id": 11681410, "id-copy": 11681410, "alias": "Wendell", "name": "WendellGarneys", "user-since": datetime("2007-07-23T13:10:29.000Z"), "user-since-copy": datetime("2007-07-23T13:10:29.000Z"), "friend-ids": {{ 11124106, 3438927, 28547601, 18074764, 35037765, 25438231, 8196141, 26000844, 6063826, 22981069, 31549929, 33158093, 40748728, 12245244, 2442169, 7879517, 877005, 24286984 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2008-02-10"), "end-date": date("2008-05-15") } ] }
+{ "id": 11708152, "id-copy": 11708152, "alias": "Gil", "name": "GilElsas", "user-since": datetime("2009-04-08T15:40:59.000Z"), "user-since-copy": datetime("2009-04-08T15:40:59.000Z"), "friend-ids": {{ 14661698, 22657473, 28892770, 39654430, 46338819, 44974094, 38564659, 24819725, 21550883, 37711934, 37285158, 20050610, 19163447, 10974750, 47513067, 43771947, 23633824 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2002-09-21"), "end-date": date("2011-03-11") } ] }
+{ "id": 11725939, "id-copy": 11725939, "alias": "Clover", "name": "CloverAlice", "user-since": datetime("2007-07-12T05:17:52.000Z"), "user-since-copy": datetime("2007-07-12T05:17:52.000Z"), "friend-ids": {{ 24426905, 6647137, 25463555, 11443041, 10549599, 35925634, 4053835, 11813301, 6976204, 26680887, 29934690, 7935338, 45092791, 30510709 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2000-05-04"), "end-date": date("2000-08-24") } ] }
+{ "id": 11729626, "id-copy": 11729626, "alias": "Kassandra", "name": "KassandraBaker", "user-since": datetime("2010-12-26T12:18:49.000Z"), "user-since-copy": datetime("2010-12-26T12:18:49.000Z"), "friend-ids": {{ 2336026, 15350108, 46098823, 35193308, 34644345, 45989141, 31179029, 15991657, 12863616, 18297246, 26571280, 16935684, 31339122, 10623785, 24666322, 23094237, 28117245, 40096052, 37538843, 8085609, 2437482, 8885815, 42016898, 4654048 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2007-07-10"), "end-date": null } ] }
+{ "id": 11741821, "id-copy": 11741821, "alias": "Cal", "name": "CalHowe", "user-since": datetime("2005-12-27T20:26:31.000Z"), "user-since-copy": datetime("2005-12-27T20:26:31.000Z"), "friend-ids": {{ 45052138 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2006-12-10"), "end-date": date("2006-02-25") } ] }
+{ "id": 11758474, "id-copy": 11758474, "alias": "Xavier", "name": "XavierAtweeke", "user-since": datetime("2011-10-03T12:35:37.000Z"), "user-since-copy": datetime("2011-10-03T12:35:37.000Z"), "friend-ids": {{ 30110740, 41016650, 23732518, 14585316, 34474077, 47591093, 10803514, 8912354, 43455040, 21960801, 31978150, 40693811, 14585416, 36411476, 20556412, 44978412, 7266670, 506620, 7686872 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2004-03-07"), "end-date": null } ] }
+{ "id": 11774587, "id-copy": 11774587, "alias": "Shari", "name": "ShariMortland", "user-since": datetime("2012-07-21T10:15:22.000Z"), "user-since-copy": datetime("2012-07-21T10:15:22.000Z"), "friend-ids": {{ 17661326, 29399532, 38328734, 38063295, 46008807, 29873254, 4407085, 27903240 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2005-05-18"), "end-date": null } ] }
+{ "id": 11780581, "id-copy": 11780581, "alias": "Simona", "name": "SimonaDrumm", "user-since": datetime("2010-09-10T00:03:56.000Z"), "user-since-copy": datetime("2010-09-10T00:03:56.000Z"), "friend-ids": {{ 14930223, 14107902, 18276584, 12824637, 44738306, 252529, 17504815, 26802467, 33312123, 15516170, 9060069, 42300993, 15746839, 61844, 1966381, 31284798, 40145954, 31282156, 15764470, 9894586, 41833755 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2011-03-27"), "end-date": null } ] }
+{ "id": 11791471, "id-copy": 11791471, "alias": "Robt", "name": "RobtChristman", "user-since": datetime("2009-08-08T21:01:18.000Z"), "user-since-copy": datetime("2009-08-08T21:01:18.000Z"), "friend-ids": {{ 9265036, 17976405, 32435071, 7236713, 21936800, 42691957, 35478695, 40052609, 14063303, 43864025, 1254074, 39237113, 11307270, 37061951, 17360733, 21102633, 21364546, 35445000, 44857867 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2008-01-02"), "end-date": date("2010-05-19") } ] }
+{ "id": 11804755, "id-copy": 11804755, "alias": "Humbert", "name": "HumbertArmitage", "user-since": datetime("2008-01-01T21:14:34.000Z"), "user-since-copy": datetime("2008-01-01T21:14:34.000Z"), "friend-ids": {{ 15498777, 1984479, 18672418, 13137212, 17931875, 10446256, 39250716, 9422828, 35469173, 35940705, 44217206 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2005-11-12"), "end-date": null } ] }
+{ "id": 11809528, "id-copy": 11809528, "alias": "Donya", "name": "DonyaNash", "user-since": datetime("2008-06-09T09:42:48.000Z"), "user-since-copy": datetime("2008-06-09T09:42:48.000Z"), "friend-ids": {{ 25365000, 20270987, 39083310, 16364767, 1960249, 39747742, 17169019, 780802, 37012712, 27956954, 35502958, 10600365, 38247667, 47815777, 25182855, 13670701, 27795853, 24952265 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2011-10-15"), "end-date": null } ] }
+{ "id": 11811079, "id-copy": 11811079, "alias": "Kenelm", "name": "KenelmKellogg", "user-since": datetime("2006-05-14T04:13:36.000Z"), "user-since-copy": datetime("2006-05-14T04:13:36.000Z"), "friend-ids": {{ 28287762, 45591894, 12026636, 34381293, 17018521, 37239852, 5735876, 8145944, 34171842, 32986088, 16537938, 20530369, 35161854, 1076550, 26081966, 35666231 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-02-03"), "end-date": null } ] }
+{ "id": 11811196, "id-copy": 11811196, "alias": "Levi", "name": "LeviVeith", "user-since": datetime("2010-04-28T03:02:38.000Z"), "user-since-copy": datetime("2010-04-28T03:02:38.000Z"), "friend-ids": {{ 24907725, 35390929, 34837809, 5881290, 28179492, 44686412, 32544180, 20478414, 15685375, 8767940, 7295427 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2004-09-01"), "end-date": null } ] }
+{ "id": 11822506, "id-copy": 11822506, "alias": "Jerrold", "name": "JerroldEwing", "user-since": datetime("2010-08-27T22:34:36.000Z"), "user-since-copy": datetime("2010-08-27T22:34:36.000Z"), "friend-ids": {{  }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2007-03-21"), "end-date": date("2008-04-26") } ] }
+{ "id": 11886532, "id-copy": 11886532, "alias": "Tel", "name": "TelGardner", "user-since": datetime("2009-10-06T10:33:32.000Z"), "user-since-copy": datetime("2009-10-06T10:33:32.000Z"), "friend-ids": {{ 37243107, 36561786, 3939621, 13531917, 7768514, 31689833, 27145019, 9462172, 40579935, 32184519, 8668855, 26137893, 5582080, 4847233, 10244448, 42634758, 34911290, 10834989, 34800551, 14109743 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2010-07-24"), "end-date": null } ] }
+{ "id": 11886856, "id-copy": 11886856, "alias": "Eldred", "name": "EldredArmstrong", "user-since": datetime("2012-02-20T10:08:40.000Z"), "user-since-copy": datetime("2012-02-20T10:08:40.000Z"), "friend-ids": {{ 5146204, 10549788, 40744824, 38277859 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2006-09-18"), "end-date": null } ] }
+{ "id": 11888530, "id-copy": 11888530, "alias": "Louis", "name": "LouisRichards", "user-since": datetime("2011-10-26T02:27:49.000Z"), "user-since-copy": datetime("2011-10-26T02:27:49.000Z"), "friend-ids": {{ 40512993, 46289399 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2000-04-18"), "end-date": date("2002-08-03") } ] }
+{ "id": 11899576, "id-copy": 11899576, "alias": "Raven", "name": "RavenAdams", "user-since": datetime("2011-12-02T12:46:45.000Z"), "user-since-copy": datetime("2011-12-02T12:46:45.000Z"), "friend-ids": {{ 33232775, 8985272, 34257645, 15577012, 3749136, 36721837, 17368752, 36931534, 30688133, 36202643, 8373322, 34639728, 10776563, 5758944, 19414939, 46764976, 29704238, 38970621, 9462886, 46724087, 29191126, 9001393 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2003-03-02"), "end-date": null } ] }
+{ "id": 11919640, "id-copy": 11919640, "alias": "Blanch", "name": "BlanchHawkins", "user-since": datetime("2007-09-24T10:11:40.000Z"), "user-since-copy": datetime("2007-09-24T10:11:40.000Z"), "friend-ids": {{ 28731986, 7289796, 42121816, 33230171 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2007-09-17"), "end-date": null } ] }
+{ "id": 11932807, "id-copy": 11932807, "alias": "Sheridan", "name": "SheridanCarr", "user-since": datetime("2009-05-17T01:39:53.000Z"), "user-since-copy": datetime("2009-05-17T01:39:53.000Z"), "friend-ids": {{ 12836351, 10066178, 40881248, 3744364, 18904729, 10238846, 27947251, 23407801, 39613208, 34468026, 20801656, 46114253, 26807188, 13084266, 27104805, 27016320, 25825154, 16782132, 29528918 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2001-09-04"), "end-date": date("2005-01-15") } ] }
+{ "id": 11945014, "id-copy": 11945014, "alias": "Lavern", "name": "LavernRahl", "user-since": datetime("2005-08-13T08:07:58.000Z"), "user-since-copy": datetime("2005-08-13T08:07:58.000Z"), "friend-ids": {{ 15127940, 37543274, 13877909, 8961585, 13712343, 38178056, 21469501, 2994082, 24368304, 33508930, 41765591, 37858577, 42295002 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2001-07-20"), "end-date": null } ] }
+{ "id": 11953306, "id-copy": 11953306, "alias": "Teale", "name": "TealeHoltzer", "user-since": datetime("2007-02-14T21:50:54.000Z"), "user-since-copy": datetime("2007-02-14T21:50:54.000Z"), "friend-ids": {{ 30902622, 26223630, 46832466, 32585590, 34005386, 23371032, 25984545, 7502619 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2010-02-14"), "end-date": date("2011-07-08") } ] }
+{ "id": 11957011, "id-copy": 11957011, "alias": "Frannie", "name": "FrannieRoose", "user-since": datetime("2007-04-05T18:00:20.000Z"), "user-since-copy": datetime("2007-04-05T18:00:20.000Z"), "friend-ids": {{ 9114095, 4905395, 41862236, 21901856, 39479601, 4025127, 1517878, 16698416, 10853001, 18625728, 15395201, 17825510, 40384476, 18779630, 1832149, 41381869, 40010653, 21121933, 18598397, 12806945, 11465558 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2006-12-22"), "end-date": null } ] }
+{ "id": 11972860, "id-copy": 11972860, "alias": "Isador", "name": "IsadorCattley", "user-since": datetime("2005-04-10T23:37:49.000Z"), "user-since-copy": datetime("2005-04-10T23:37:49.000Z"), "friend-ids": {{ 39841874, 9405322, 3110197, 39455453, 11331432, 31809217, 45852118, 12899824, 19561127, 3413313, 19872192, 13427579, 140732, 6913603 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2006-01-01"), "end-date": date("2009-11-22") } ] }
+{ "id": 11978782, "id-copy": 11978782, "alias": "Louiza", "name": "LouizaLlora", "user-since": datetime("2012-06-24T06:19:05.000Z"), "user-since-copy": datetime("2012-06-24T06:19:05.000Z"), "friend-ids": {{ 36495107, 35125435, 30347420, 17703387, 40909002 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2008-05-25"), "end-date": null } ] }
+{ "id": 11996683, "id-copy": 11996683, "alias": "Ivy", "name": "IvyReddish", "user-since": datetime("2008-10-09T09:54:46.000Z"), "user-since-copy": datetime("2008-10-09T09:54:46.000Z"), "friend-ids": {{ 42344158, 40312093, 15782003 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2003-04-16"), "end-date": null } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/caret0/caret0.1.adm b/asterix-app/src/test/resources/runtimets/results/numeric/caret0/caret0.1.adm
new file mode 100644
index 0000000..1edec8a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/caret0/caret0.1.adm
@@ -0,0 +1 @@
+{ "c1": Infinityd, "c2": 1.6777216E7d, "c3": 9 }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/query-issue355/query-issue355.1.adm b/asterix-app/src/test/resources/runtimets/results/numeric/query-issue355/query-issue355.1.adm
new file mode 100644
index 0000000..8b13789
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/numeric/query-issue355/query-issue355.1.adm
@@ -0,0 +1 @@
+
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue377/query-issue377.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue377/query-issue377.1.adm
index c3bb80f..1a7caf5 100644
--- a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue377/query-issue377.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue377/query-issue377.1.adm
@@ -1,23 +1,18 @@
 { "id": 9142198, "similar-users": [  ], "name": "SherryFea" }
 { "id": 9313492, "similar-users": [  ], "name": "TeraWolfe" }
 { "id": 9478720, "similar-users": [  ], "name": "AngeliaKettlewell" }
-{ "id": 10001080, "similar-users": [  ], "name": "GarrettBode" }
-{ "id": 10179538, "similar-users": [  ], "name": "OrlandoBaxter" }
-{ "id": 10307032, "similar-users": [  ], "name": "QuentinSauter" }
-{ "id": 10394488, "similar-users": [  ], "name": "OswaldRay" }
-{ "id": 10423588, "similar-users": [  ], "name": "ShirleneRuch" }
-{ "id": 10495420, "similar-users": [  ], "name": "WendyMcloskey" }
-{ "id": 11307946, "similar-users": [  ], "name": "HelgaStough" }
-{ "id": 11447332, "similar-users": [  ], "name": "SherisseMaugham" }
-{ "id": 11570326, "similar-users": [  ], "name": "LindenFilby" }
-{ "id": 11951098, "similar-users": [  ], "name": "TeraByers" }
-{ "id": 11954992, "similar-users": [  ], "name": "CaitlinLangston" }
 { "id": 9510451, "similar-users": [  ], "name": "ChuckFinck" }
 { "id": 9594523, "similar-users": [  ], "name": "TamWillcox" }
 { "id": 9629395, "similar-users": [  ], "name": "JuliusWire" }
 { "id": 9988417, "similar-users": [  ], "name": "ColineLane" }
+{ "id": 10001080, "similar-users": [  ], "name": "GarrettBode" }
+{ "id": 10179538, "similar-users": [  ], "name": "OrlandoBaxter" }
 { "id": 10272571, "similar-users": [  ], "name": "JarrettGoldvogel" }
+{ "id": 10307032, "similar-users": [  ], "name": "QuentinSauter" }
 { "id": 10361965, "similar-users": [  ], "name": "ArlenFlick" }
+{ "id": 10394488, "similar-users": [  ], "name": "OswaldRay" }
+{ "id": 10423588, "similar-users": [  ], "name": "ShirleneRuch" }
+{ "id": 10495420, "similar-users": [  ], "name": "WendyMcloskey" }
 { "id": 10498285, "similar-users": [  ], "name": "KileyBridger" }
 { "id": 10733617, "similar-users": [  ], "name": "LeonardoKight" }
 { "id": 10874791, "similar-users": [  ], "name": "HaydeeGarratt" }
@@ -25,5 +20,10 @@
 { "id": 11061631, "similar-users": [  ], "name": "MaxeneKellogg" }
 { "id": 11068231, "similar-users": [  ], "name": "DinahSwink" }
 { "id": 11140213, "similar-users": [  ], "name": "MontgomeryWhittier" }
+{ "id": 11307946, "similar-users": [  ], "name": "HelgaStough" }
 { "id": 11381089, "similar-users": [  ], "name": "EarleneAmmons" }
+{ "id": 11447332, "similar-users": [  ], "name": "SherisseMaugham" }
+{ "id": 11570326, "similar-users": [  ], "name": "LindenFilby" }
 { "id": 11675221, "similar-users": [  ], "name": "CalantheGearhart" }
+{ "id": 11951098, "similar-users": [  ], "name": "TeraByers" }
+{ "id": 11954992, "similar-users": [  ], "name": "CaitlinLangston" }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue423-2/query-issue423-2.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue423-2/query-issue423-2.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue423-2/query-issue423-2.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue456/query-issue456.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue456/query-issue456.1.adm
new file mode 100644
index 0000000..c989cc7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue456/query-issue456.1.adm
@@ -0,0 +1,2 @@
+[ 1, 8i64 ]
+[ 2, 8i64 ]
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue465/query-issue465.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue465/query-issue465.1.adm
new file mode 100644
index 0000000..15c3035
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue465/query-issue465.1.adm
@@ -0,0 +1 @@
+[ { "r1": 1234 }, { "r2": 456 } ]
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue487/query-issue487.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue487/query-issue487.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue487/query-issue487.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/string/string-to-codepoint2/string-to-codepoint2.1.adm b/asterix-app/src/test/resources/runtimets/results/string/string-to-codepoint2/string-to-codepoint2.1.adm
new file mode 100644
index 0000000..eacdb11
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/string/string-to-codepoint2/string-to-codepoint2.1.adm
@@ -0,0 +1 @@
+{ "result1": [ 27426, 36814 ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/string/substr04/substr04.1.adm b/asterix-app/src/test/resources/runtimets/results/string/substr04/substr04.1.adm
index fcd3396..74bd0d6 100644
--- a/asterix-app/src/test/resources/runtimets/results/string/substr04/substr04.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/string/substr04/substr04.1.adm
@@ -5,6 +5,4 @@
 "ABCD"
 "Irvine"
 "UC Irvine"
-"UC Irvine"
-"Irvine"
 "Irvine"
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.1.adm b/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.1.adm
new file mode 100644
index 0000000..4d1cf59
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.1.adm
@@ -0,0 +1,3 @@
+{ "date": date("-9971-09-24"), "time": time("11:38:17.154Z"), "datetime": datetime("1259-11-13T09:49:11.852Z"), "duration": duration("P473653Y9M4566143DT10H20M53.61S"), "year-month-duration": year-month-duration("P148233Y10M"), "day-time-duration": day-time-duration("-P7236357DT2H56M56.164S"), "date-interval": interval-date("-0255-09-06, 4925-05-03"), "time-interval": interval-time("23:10:45.169Z, 01:37:48.736Z"), "datetime-interval": interval-datetime("0534-12-08T08:20:31.487Z, 6778-02-16T22:40:21.653Z") }
+{ "date": date("4619-11-23"), "time": time("14:29:36.786Z"), "datetime": datetime("2749-01-27T17:27:30.020Z"), "duration": duration("-P474133Y7M854630DT4H40M6.45S"), "year-month-duration": year-month-duration("P193989Y3M"), "day-time-duration": day-time-duration("P4477686DT4H49M31.87S"), "date-interval": interval-date("-9537-08-04, 9656-06-03"), "time-interval": interval-time("12:04:45.689Z, 12:41:59.002Z"), "datetime-interval": interval-datetime("-2640-10-11T17:32:15.675Z, 4104-02-01T05:59:11.902Z") }
+{ "date": date("7986-11-25"), "time": time("12:49:39.736Z"), "datetime": datetime("-8337-01-30T15:23:07.598Z"), "duration": duration("-P184484Y7M2241423DT10H42M49.500S"), "year-month-duration": year-month-duration("-P546031Y3M"), "day-time-duration": day-time-duration("P2623386DT10H32M31.983S"), "date-interval": interval-date("-4514-05-24, 3337-08-26"), "time-interval": interval-time("04:16:42.321Z, 02:22:56.816Z"), "datetime-interval": interval-datetime("2129-12-12T13:18:35.758Z, 8647-07-01T13:10:19.691Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/.tinysocial-suite.5.adm.swp b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/.tinysocial-suite.5.adm.swp
new file mode 100644
index 0000000..e7eb7a1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/.tinysocial-suite.5.adm.swp
Binary files differ
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.10.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.10.adm
new file mode 100644
index 0000000..5451c76
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.10.adm
@@ -0,0 +1,12 @@
+{ "message": " can't stand iphone its platform is terrible", "nearby-messages": [ { "msgtxt": " can't stand iphone its platform is terrible" } ] }
+{ "message": " can't stand motorola its speed is terrible:(", "nearby-messages": [ { "msgtxt": " can't stand motorola its speed is terrible:(" } ] }
+{ "message": " hate verizon its voice-clarity is OMG:(", "nearby-messages": [ { "msgtxt": " hate verizon its voice-clarity is OMG:(" }, { "msgtxt": " like motorola the speed is good:)" } ] }
+{ "message": " like iphone the voice-clarity is good:)", "nearby-messages": [ { "msgtxt": " like iphone the voice-clarity is good:)" } ] }
+{ "message": " like motorola the speed is good:)", "nearby-messages": [ { "msgtxt": " hate verizon its voice-clarity is OMG:(" }, { "msgtxt": " like motorola the speed is good:)" } ] }
+{ "message": " like samsung the platform is good", "nearby-messages": [ { "msgtxt": " like samsung the platform is good" } ] }
+{ "message": " like samsung the voice-command is amazing:)", "nearby-messages": [ { "msgtxt": " like samsung the voice-command is amazing:)" } ] }
+{ "message": " like sprint the voice-command is mind-blowing:)", "nearby-messages": [ { "msgtxt": " like sprint the voice-command is mind-blowing:)" } ] }
+{ "message": " like t-mobile the shortcut-menu is awesome:)", "nearby-messages": [ { "msgtxt": " like t-mobile the shortcut-menu is awesome:)" } ] }
+{ "message": " like verizon its shortcut-menu is awesome:)", "nearby-messages": [ { "msgtxt": " like verizon its shortcut-menu is awesome:)" } ] }
+{ "message": " love t-mobile its customization is good:)", "nearby-messages": [ { "msgtxt": " love t-mobile its customization is good:)" } ] }
+{ "message": " love verizon its voicemail-service is awesome", "nearby-messages": [ { "msgtxt": " love verizon its voicemail-service is awesome" } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.11.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.11.adm
new file mode 100644
index 0000000..a5ec171
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.11.adm
@@ -0,0 +1,10 @@
+{ "id": 1, "name": "MargaritaStoddard", "similar-users": [  ] }
+{ "id": 2, "name": "IsbelDull", "similar-users": [  ] }
+{ "id": 3, "name": "EmoryUnk", "similar-users": [  ] }
+{ "id": 4, "name": "NicholasStroh", "similar-users": [  ] }
+{ "id": 5, "name": "VonKemble", "similar-users": [  ] }
+{ "id": 6, "name": "WillisWynne", "similar-users": [  ] }
+{ "id": 7, "name": "SuzannaTillson", "similar-users": [  ] }
+{ "id": 8, "name": "NilaMilliron", "similar-users": [ { "twitter-screenname": "NilaMilliron_tw", "twitter-name": "Nila Milliron" } ] }
+{ "id": 9, "name": "WoodrowNehling", "similar-users": [  ] }
+{ "id": 10, "name": "BramHatch", "similar-users": [  ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.12.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.12.adm
new file mode 100644
index 0000000..975222c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.12.adm
@@ -0,0 +1,7 @@
+{ "id": 1, "alias": "Margarita", "name": "MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000Z"), "friend-ids": {{ 2, 3, 6, 10 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-08-06"), "end-date": null } ] }
+{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27"), "end-date": null } ] }
+{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08"), "end-date": null } ] }
+{ "id": 5, "alias": "Von", "name": "VonKemble", "user-since": datetime("2010-01-05T10:10:00.000Z"), "friend-ids": {{ 3, 6, 10 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-11-27"), "end-date": null } ] }
+{ "id": 6, "alias": "Willis", "name": "WillisWynne", "user-since": datetime("2005-01-17T10:10:00.000Z"), "friend-ids": {{ 1, 3, 7 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2009-05-15"), "end-date": null } ] }
+{ "id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000Z"), "friend-ids": {{ 6 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2011-04-19"), "end-date": null } ] }
+{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000Z"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28"), "end-date": null } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.13.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.13.adm
new file mode 100644
index 0000000..2df77f6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.13.adm
@@ -0,0 +1,3 @@
+{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+{ "id": 9, "alias": "Woodrow", "name": "WoodrowNehling", "user-since": datetime("2005-09-20T10:10:00.000Z"), "friend-ids": {{ 3, 10 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2003-04-22"), "end-date": date("2009-12-13") } ] }
+{ "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.14.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.14.adm
new file mode 100644
index 0000000..125b90b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.14.adm
@@ -0,0 +1 @@
+10i64
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.15.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.15.adm
new file mode 100644
index 0000000..084aa59
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.15.adm
@@ -0,0 +1,5 @@
+{ "user": "ChangEwing_573", "count": 1i64 }
+{ "user": "ColineGeyer@63", "count": 3i64 }
+{ "user": "NathanGiesen@211", "count": 6i64 }
+{ "user": "NilaMilliron_tw", "count": 1i64 }
+{ "user": "OliJackson_512", "count": 1i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.16.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.16.adm
new file mode 100644
index 0000000..084aa59
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.16.adm
@@ -0,0 +1,5 @@
+{ "user": "ChangEwing_573", "count": 1i64 }
+{ "user": "ColineGeyer@63", "count": 3i64 }
+{ "user": "NathanGiesen@211", "count": 6i64 }
+{ "user": "NilaMilliron_tw", "count": 1i64 }
+{ "user": "OliJackson_512", "count": 1i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.17.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.17.adm
new file mode 100644
index 0000000..689c29d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.17.adm
@@ -0,0 +1,3 @@
+{ "user": "OliJackson_512", "count": 1i64 }
+{ "user": "NilaMilliron_tw", "count": 1i64 }
+{ "user": "ChangEwing_573", "count": 1i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.18.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.18.adm
new file mode 100644
index 0000000..b59a814
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.18.adm
@@ -0,0 +1,12 @@
+{ "tweet": { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, "similar-tweets": [ {{ "t-mobile", "shortcut-menu" }} ] }
+{ "tweet": { "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, "similar-tweets": [ {{ "verizon", "shortcut-menu" }}, {{ "iphone", "voice-clarity" }}, {{ "verizon", "voicemail-service" }} ] }
+{ "tweet": { "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, "similar-tweets": [ {{ "iphone", "voice-clarity" }}, {{ "samsung", "platform" }} ] }
+{ "tweet": { "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, "similar-tweets": [ {{ "sprint", "voice-command" }}, {{ "samsung", "platform" }} ] }
+{ "tweet": { "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "t-mobile", "shortcut-menu" }}, {{ "verizon", "voicemail-service" }} ] }
+{ "tweet": { "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
+{ "tweet": { "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, "similar-tweets": [ {{ "samsung", "voice-command" }} ] }
+{ "tweet": { "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
+{ "tweet": { "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "iphone", "platform" }} ] }
+{ "tweet": { "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, "similar-tweets": [ {{ "iphone", "platform" }}, {{ "samsung", "voice-command" }} ] }
+{ "tweet": { "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "t-mobile", "customization" }}, {{ "verizon", "shortcut-menu" }} ] }
+{ "tweet": { "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "verizon", "shortcut-menu" }} ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.20.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.20.adm
new file mode 100644
index 0000000..5c8c013
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.20.adm
@@ -0,0 +1,13 @@
+{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
+{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
+{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39345, "statuses_count": 479, "name": "Nathan Giesen", "followers_count": 49420 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:35.000Z"), "referred-topics": {{ "tweeting" }}, "message-text": "tweety tweet, my fellow tweeters!" }
+{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.22.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.22.adm
new file mode 100644
index 0000000..2dd16db
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.22.adm
@@ -0,0 +1 @@
+0i64
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.4.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.4.adm
new file mode 100644
index 0000000..e6d8392
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.4.adm
@@ -0,0 +1 @@
+{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000Z"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28"), "end-date": null } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.5.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.5.adm
new file mode 100644
index 0000000..5e2b105
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.5.adm
@@ -0,0 +1,3 @@
+{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27"), "end-date": null } ] }
+{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08"), "end-date": null } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.6.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.6.adm
new file mode 100644
index 0000000..dba1e35
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.6.adm
@@ -0,0 +1,4 @@
+{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27"), "end-date": null } ] }
+{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08"), "end-date": null } ] }
+{ "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.7.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.7.adm
new file mode 100644
index 0000000..a403f1b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.7.adm
@@ -0,0 +1,15 @@
+{ "uname": "BramHatch", "message": " can't stand t-mobile its voicemail-service is OMG:(" }
+{ "uname": "BramHatch", "message": " dislike iphone the voice-command is bad:(" }
+{ "uname": "EmoryUnk", "message": " love sprint its shortcut-menu is awesome:)" }
+{ "uname": "EmoryUnk", "message": " love verizon its wireless is good" }
+{ "uname": "IsbelDull", "message": " like samsung the plan is amazing" }
+{ "uname": "IsbelDull", "message": " like t-mobile its platform is mind-blowing" }
+{ "uname": "MargaritaStoddard", "message": " dislike iphone its touch-screen is horrible" }
+{ "uname": "MargaritaStoddard", "message": " can't stand at&t the network is horrible:(" }
+{ "uname": "MargaritaStoddard", "message": " like verizon the 3G is awesome:)" }
+{ "uname": "MargaritaStoddard", "message": " can't stand motorola the touch-screen is terrible" }
+{ "uname": "MargaritaStoddard", "message": " can't stand at&t its plan is terrible" }
+{ "uname": "SuzannaTillson", "message": " like iphone the voicemail-service is awesome" }
+{ "uname": "VonKemble", "message": " dislike sprint the speed is horrible" }
+{ "uname": "WillisWynne", "message": " love sprint the customization is mind-blowing" }
+{ "uname": "WoodrowNehling", "message": " love at&t its 3G is good:)" }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.8.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.8.adm
new file mode 100644
index 0000000..a403f1b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.8.adm
@@ -0,0 +1,15 @@
+{ "uname": "BramHatch", "message": " can't stand t-mobile its voicemail-service is OMG:(" }
+{ "uname": "BramHatch", "message": " dislike iphone the voice-command is bad:(" }
+{ "uname": "EmoryUnk", "message": " love sprint its shortcut-menu is awesome:)" }
+{ "uname": "EmoryUnk", "message": " love verizon its wireless is good" }
+{ "uname": "IsbelDull", "message": " like samsung the plan is amazing" }
+{ "uname": "IsbelDull", "message": " like t-mobile its platform is mind-blowing" }
+{ "uname": "MargaritaStoddard", "message": " dislike iphone its touch-screen is horrible" }
+{ "uname": "MargaritaStoddard", "message": " can't stand at&t the network is horrible:(" }
+{ "uname": "MargaritaStoddard", "message": " like verizon the 3G is awesome:)" }
+{ "uname": "MargaritaStoddard", "message": " can't stand motorola the touch-screen is terrible" }
+{ "uname": "MargaritaStoddard", "message": " can't stand at&t its plan is terrible" }
+{ "uname": "SuzannaTillson", "message": " like iphone the voicemail-service is awesome" }
+{ "uname": "VonKemble", "message": " dislike sprint the speed is horrible" }
+{ "uname": "WillisWynne", "message": " love sprint the customization is mind-blowing" }
+{ "uname": "WoodrowNehling", "message": " love at&t its 3G is good:)" }
diff --git a/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.9.adm b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.9.adm
new file mode 100644
index 0000000..09b0f08
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.9.adm
@@ -0,0 +1,10 @@
+{ "uname": "BramHatch", "messages": [ " can't stand t-mobile its voicemail-service is OMG:(", " dislike iphone the voice-command is bad:(" ] }
+{ "uname": "EmoryUnk", "messages": [ " love sprint its shortcut-menu is awesome:)", " love verizon its wireless is good" ] }
+{ "uname": "IsbelDull", "messages": [ " like samsung the plan is amazing", " like t-mobile its platform is mind-blowing" ] }
+{ "uname": "MargaritaStoddard", "messages": [ " dislike iphone its touch-screen is horrible", " can't stand at&t the network is horrible:(", " like verizon the 3G is awesome:)", " can't stand motorola the touch-screen is terrible", " can't stand at&t its plan is terrible" ] }
+{ "uname": "NicholasStroh", "messages": [  ] }
+{ "uname": "NilaMilliron", "messages": [  ] }
+{ "uname": "SuzannaTillson", "messages": [ " like iphone the voicemail-service is awesome" ] }
+{ "uname": "VonKemble", "messages": [ " dislike sprint the speed is horrible" ] }
+{ "uname": "WillisWynne", "messages": [ " love sprint the customization is mind-blowing" ] }
+{ "uname": "WoodrowNehling", "messages": [ " love at&t its 3G is good:)" ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue172/query-issue172.1.adm b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue172/query-issue172.1.adm
new file mode 100644
index 0000000..aabe6ec
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue172/query-issue172.1.adm
@@ -0,0 +1 @@
+21
diff --git a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue244/query-issue244.1.adm b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue244/query-issue244.1.adm
new file mode 100644
index 0000000..92f6371
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue244/query-issue244.1.adm
@@ -0,0 +1,5 @@
+{ "id": 16, "name": "Sam" }
+{ "id": 21, "name": "John" }
+{ "id": 34, "name": "Bill" }
+{ "id": 41, "name": "Joy" }
+{ "id": 67, "name": "Ravi" }
diff --git a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue489/query-issue489.2.adm b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue489/query-issue489.2.adm
new file mode 100644
index 0000000..06e4a82
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue489/query-issue489.2.adm
@@ -0,0 +1 @@
+2i64
diff --git a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue489/query-issue489.4.adm b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue489/query-issue489.4.adm
new file mode 100644
index 0000000..2dd16db
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-issue489/query-issue489.4.adm
@@ -0,0 +1 @@
+0i64
diff --git a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
index 8bd28d4..c42d848 100644
--- a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
@@ -1,6 +1,6 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "DMLTest", "DatasetName": "FacebookUsers1", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:32:02 PDT 2013", "DatasetId": 366, "PendingOp": 0 }
+{ "DataverseName": "DMLTest", "DatasetName": "FacebookUsers2", "DataTypeName": "FacebookUserType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:32:02 PDT 2013", "DatasetId": 367, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:30:56 PDT 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:30:57 PDT 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:30:57 PDT 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Jun 03 02:30:56 PDT 2013", "DatasetId": 1, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 9e6c308..9711930 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -1040,16 +1040,6 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
-      <compilation-unit name="scan-insert-inverted-index-fuzzy-ngram-secondary-index">
-        <output-dir compare="Text">scan-insert-inverted-index-fuzzy-ngram-secondary-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="dml">
-      <compilation-unit name="scan-insert-inverted-index-fuzzy-word-secondary-index">
-        <output-dir compare="Text">scan-insert-inverted-index-fuzzy-word-secondary-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="dml">
       <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable">
         <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
       </compilation-unit>
@@ -1060,16 +1050,6 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
-      <compilation-unit name="scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable">
-        <output-dir compare="Text">scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="dml">
-      <compilation-unit name="scan-insert-inverted-index-fuzzy-word-secondary-index-nullable">
-        <output-dir compare="Text">scan-insert-inverted-index-fuzzy-word-secondary-index-nullable</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="dml">
       <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index">
         <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
       </compilation-unit>
@@ -1080,16 +1060,6 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
-      <compilation-unit name="scan-delete-inverted-index-fuzzy-ngram-secondary-index">
-        <output-dir compare="Text">scan-delete-inverted-index-fuzzy-ngram-secondary-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="dml">
-      <compilation-unit name="scan-delete-inverted-index-fuzzy-word-secondary-index">
-        <output-dir compare="Text">scan-delete-inverted-index-fuzzy-word-secondary-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="dml">
       <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-nullable">
         <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
       </compilation-unit>
@@ -1099,16 +1069,6 @@
         <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
       </compilation-unit>
     </test-case>
-    <test-case FilePath="dml">
-      <compilation-unit name="scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable">
-        <output-dir compare="Text">scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="dml">
-      <compilation-unit name="scan-delete-inverted-index-fuzzy-word-secondary-index-nullable">
-        <output-dir compare="Text">scan-delete-inverted-index-fuzzy-word-secondary-index-nullable</output-dir>
-      </compilation-unit>
-    </test-case>
   </test-group>
   <test-group name="employee">
     <test-case FilePath="employee">
@@ -1790,6 +1750,11 @@
       </compilation-unit>
     </test-case>
     -->
+    <test-case FilePath="fuzzyjoin">
+      <compilation-unit name="opentype">
+        <output-dir compare="Text">opentype</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="index-join">
     <test-case FilePath="index-join">
@@ -1830,56 +1795,6 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
-      <compilation-unit name="fuzzy-inverted-index-ngram-contains">
-        <output-dir compare="Text">fuzzy-inverted-index-ngram-contains</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="index-selection">
-      <compilation-unit name="fuzzy-inverted-index-ngram-edit-distance-panic">
-        <output-dir compare="Text">fuzzy-inverted-index-ngram-edit-distance-panic</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="index-selection">
-      <compilation-unit name="fuzzy-inverted-index-ngram-edit-distance">
-        <output-dir compare="Text">fuzzy-inverted-index-ngram-edit-distance</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="index-selection">
-      <compilation-unit name="fuzzy-inverted-index-ngram-jaccard">
-        <output-dir compare="Text">fuzzy-inverted-index-ngram-jaccard</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="index-selection">
-      <compilation-unit name="fuzzy-inverted-index-olist-edit-distance-panic">
-        <output-dir compare="Text">fuzzy-inverted-index-olist-edit-distance-panic</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="index-selection">
-      <compilation-unit name="fuzzy-inverted-index-olist-edit-distance">
-        <output-dir compare="Text">fuzzy-inverted-index-olist-edit-distance</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="index-selection">
-      <compilation-unit name="fuzzy-inverted-index-olist-jaccard">
-        <output-dir compare="Text">fuzzy-inverted-index-olist-jaccard</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="index-selection">
-      <compilation-unit name="fuzzy-inverted-index-ulist-jaccard">
-        <output-dir compare="Text">fuzzy-inverted-index-ulist-jaccard</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="index-selection">
-      <compilation-unit name="fuzzy-inverted-index-word-contains">
-        <output-dir compare="Text">fuzzy-inverted-index-word-contains</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="index-selection">
-      <compilation-unit name="fuzzy-inverted-index-word-jaccard">
-        <output-dir compare="Text">fuzzy-inverted-index-word-jaccard</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="index-selection">
       <compilation-unit name="inverted-index-ngram-contains">
         <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
       </compilation-unit>
@@ -2199,6 +2114,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="flushtest">
+        <output-dir compare="Text">flushtest</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="groupby-orderby-count">
         <output-dir compare="Text">groupby-orderby-count</output-dir>
       </compilation-unit>
@@ -2245,6 +2165,11 @@
   </test-group>
   <test-group name="numeric">
     <test-case FilePath="numeric">
+      <compilation-unit name="caret0">
+        <output-dir compare="Text">caret0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
       <compilation-unit name="abs0">
         <output-dir compare="Text">abs0</output-dir>
       </compilation-unit>
@@ -2554,6 +2479,12 @@
         <output-dir compare="Text">unary-minus_null</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="query-issue355">
+        <output-dir compare="Text">query-issue355</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="open-closed">
     <!--
@@ -2792,12 +2723,22 @@
     </test-case>
     <test-case FilePath="open-closed">
       <compilation-unit name="query-issue453">
-	<output-dir compare="Text">query-issue453</output-dir>
+        <output-dir compare="Text">query-issue453</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="open-closed">
       <compilation-unit name="query-issue453-2">
-	<output-dir compare="Text">query-issue453-2</output-dir>
+        <output-dir compare="Text">query-issue453-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue456"> 
+        <output-dir compare="Text">query-issue456</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue465"> 
+        <output-dir compare="Text">query-issue465</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="open-closed">
@@ -2806,11 +2747,27 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue423">
+        <output-dir compare="Text">query-issue423</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue423-2">
+        <output-dir compare="Text">query-issue423-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
       <compilation-unit name="query-issue442">
         <output-dir compare="Text">query-issue442</output-dir>
         <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue487">
+        <output-dir compare="Text">query-issue487</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="quantifiers">
     <test-case FilePath="quantifiers">
@@ -3557,6 +3514,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="string">
+      <compilation-unit name="string-to-codepoint2">
+        <output-dir compare="Text">string-to-codepoint2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
       <compilation-unit name="strlen02">
         <output-dir compare="Text">strlen02</output-dir>
       </compilation-unit>
@@ -3586,7 +3548,6 @@
         <output-dir compare="Text">substr01</output-dir>
       </compilation-unit>
     </test-case>
-    <!-- Issue no 219
     <test-case FilePath="string">
       <compilation-unit name="substr04">
         <output-dir compare="Text">substr04</output-dir>
@@ -3597,7 +3558,6 @@
         <output-dir compare="Text">substr05</output-dir>
       </compilation-unit>
     </test-case>
-    -->
     <test-case FilePath="string">
       <compilation-unit name="substr06">
         <output-dir compare="Text">substr06</output-dir>
@@ -4016,12 +3976,22 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue172">
+        <output-dir compare="Text">query-issue172</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
       <compilation-unit name="query-issue455">
         <output-dir compare="Text">query-issue455</output-dir>
         <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue489">
+        <output-dir compare="Text">query-issue489</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
       <compilation-unit name="udf01">
         <output-dir compare="Text">udf01</output-dir>
       </compilation-unit>
@@ -4212,6 +4182,11 @@
         <output-dir compare="Text">issue289_query</output-dir>
       </compilation-unit>
     </test-case>
+     <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue244">
+        <output-dir compare="Text">query-issue244</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="hints">
     <test-case FilePath="hints">
@@ -4322,6 +4297,11 @@
         <output-dir compare="Text">insert_from_ext_ds</output-dir>
       </compilation-unit>
   	</test-case>
+   <test-case FilePath="temporal">
+    <compilation-unit name="insert_from_ext_ds_2">
+        <output-dir compare="Text">insert_from_ext_ds_2</output-dir>
+      </compilation-unit>
+   </test-case>
   	<test-case FilePath="temporal">
   		<compilation-unit name="interval_functions">
         <output-dir compare="Text">interval_functions</output-dir>
@@ -4367,4 +4347,11 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="tinysocial">
+    <test-case FilePath="tinysocial">
+      <compilation-unit name="tinysocial-suite">
+        <output-dir compare="Text">tinysocial-suite</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
 </test-suite>
diff --git a/asterix-aql/.gitignore b/asterix-aql/.gitignore
new file mode 100644
index 0000000..ea8c4bf
--- /dev/null
+++ b/asterix-aql/.gitignore
@@ -0,0 +1 @@
+/target
diff --git a/asterix-aql/pom.xml b/asterix-aql/pom.xml
index 249299c..f8c652f 100644
--- a/asterix-aql/pom.xml
+++ b/asterix-aql/pom.xml
@@ -30,6 +30,7 @@
 						</goals>
 						<configuration>
 							<isStatic>false</isStatic>
+							<javaUnicodeEscape>true</javaUnicodeEscape>
 						</configuration>
 					</execution>
 					<execution>
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
index 7969941..ae5f3ef 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
@@ -155,7 +155,8 @@
         if (expression == null) {
             return;
         }
-
+        String value = metadataProvider.getConfig().get(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS);
+        boolean includePrivateFunctions = (value != null) ? Boolean.valueOf(value.toLowerCase()) : false;
         Set<FunctionSignature> functionCalls = getFunctionCalls(expression);
         for (FunctionSignature signature : functionCalls) {
 
@@ -172,9 +173,7 @@
                     buildOtherUdfs(functionDecl.getFuncBody(), functionDecls, declaredFunctions);
                 }
             } else {
-                String value = metadataProvider.getConfig().get(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS);
-                boolean includePrivateFunctions = (value != null) ? Boolean.valueOf(value.toLowerCase()) : false;
-                if (isBuiltinFunction(signature, includePrivateFunctions)) {
+                if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(signature, includePrivateFunctions)) {
                     continue;
                 } else {
                     throw new AsterixException(" unknown function " + signature);
@@ -195,31 +194,6 @@
 
     }
 
-    private boolean isBuiltinFunction(FunctionSignature signature, boolean includePrivateFunctions) {
-        signature.setNamespace(AsterixBuiltinFunctions.FunctionNamespace.ASTERIX_PUBLIC.name());
-        if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(signature.getNamespace(),
-                signature.getName(), signature.getArity()))) {
-            return true;
-        }
-
-        if (includePrivateFunctions) {
-            signature.setNamespace(AlgebricksBuiltinFunctions.ALGEBRICKS_NS);
-            if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(signature.getNamespace(),
-                    signature.getName(), signature.getArity()))) {
-                return true;
-            }
-
-            signature.setNamespace(AsterixBuiltinFunctions.FunctionNamespace.ASTERIX_PRIVATE.name());
-            if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(signature.getNamespace(),
-                    signature.getName(), signature.getArity()))) {
-                return true;
-            }
-        }
-
-        return false;
-
-    }
-
     private Set<FunctionSignature> getFunctionCalls(Expression expression) throws AsterixException {
         Map<AsterixFunction, DfsColor> color = new HashMap<AsterixFunction, DfsColor>();
         Map<AsterixFunction, List<AsterixFunction>> arcs = new HashMap<AsterixFunction, List<AsterixFunction>>();
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index 2a5f534..bd21581 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -376,20 +376,10 @@
     }
   | "keyword"
     {
-      type = IndexType.WORD_INVIX;
-    }
-  | "fuzzy keyword"
-    {
       type = IndexType.FUZZY_WORD_INVIX;
     }
   | "ngram" <LEFTPAREN> <INTEGER_LITERAL>
     {
-      type = IndexType.NGRAM_INVIX;
-      gramLength = Integer.valueOf(token.image);
-    }
-  <RIGHTPAREN>
-  | "fuzzy ngram" <LEFTPAREN> <INTEGER_LITERAL>
-    {
       type = IndexType.FUZZY_NGRAM_INVIX;
       gramLength = Integer.valueOf(token.image);
     }
@@ -1713,49 +1703,34 @@
 
 Expression DatasetAccessExpression() throws ParseException:
 {
-  CallExpr callExpr;
-  List<Expression> argList = new ArrayList<Expression>();
   String funcName;
-  String dataverse;
   String arg1 = null;
   String arg2 = null;
-  LiteralExpr ds;
   Expression nameArg;
-  int arity = 0;
 }
 {  
   <DATASET>
     {
-      dataverse = MetadataConstants.METADATA_DATAVERSE_NAME;
       funcName = token.image;
     }
   ( ( arg1 = Identifier() ( "." arg2 = Identifier() )? ) 
     {
       String name = arg2 == null ? arg1 : arg1 + "." + arg2;
-      ds = new LiteralExpr();
+      LiteralExpr ds = new LiteralExpr();
       ds.setValue( new StringLiteral(name) );
-      argList.add(ds);
-      arity ++;
+      nameArg = ds;
     }
-  | ( <LEFTPAREN> nameArg = Expression()
+  | ( <LEFTPAREN> nameArg = Expression() <RIGHTPAREN> ) )  
     {
+      String dataverse = MetadataConstants.METADATA_DATAVERSE_NAME;
+      FunctionSignature signature = lookupFunctionSignature(dataverse, funcName, 1);
+      if (signature == null) {
+        signature = new FunctionSignature(dataverse, funcName, 1);
+      }
+      List<Expression> argList = new ArrayList<Expression>();
       argList.add(nameArg);
-      arity ++;
+      return new CallExpr(signature, argList);
     }
-  ( "," nameArg = Expression()
-    {
-      argList.add(nameArg);
-      arity++;
-    }
-  )* <RIGHTPAREN> ) )  
-   {
-     FunctionSignature signature = lookupFunctionSignature(dataverse, funcName, arity);
-     if (signature == null) {
-       signature = new FunctionSignature(dataverse, funcName, arity);
-     }
-     callExpr = new CallExpr(signature,argList);
-     return callExpr;
-   }
 }
 
 Expression ParenthesizedExpression() throws ParseException:
@@ -1835,21 +1810,14 @@
 	extendCurrentScope();
 }
 {
-    "for" varExp = Variable()
-    {
-     	getCurrentScope().addNewVarSymbolToScope(varExp.getVar());
-	}
-	("at" varPos = Variable()
-	  {
-	     getCurrentScope().addNewVarSymbolToScope(varPos.getVar());
-	  } 
-	 )? 
-      "in" ( inExp = Expression() )
+    "for" varExp = Variable() ("at" varPos = Variable())?  "in" ( inExp = Expression() )
     {
       fc.setVarExpr(varExp);
+      getCurrentScope().addNewVarSymbolToScope(varExp.getVar());
       fc.setInExpr(inExp);
       if (varPos != null) {
         fc.setPosExpr(varPos);
+	    getCurrentScope().addNewVarSymbolToScope(varPos.getVar());
       }
       return fc;
     }
diff --git a/asterix-common/.gitignore b/asterix-common/.gitignore
index ea8c4bf..19f2e00 100644
--- a/asterix-common/.gitignore
+++ b/asterix-common/.gitignore
@@ -1 +1,2 @@
 /target
+/target
diff --git a/asterix-common/pom.xml b/asterix-common/pom.xml
index 1a0e782..cba478c 100644
--- a/asterix-common/pom.xml
+++ b/asterix-common/pom.xml
@@ -99,13 +99,6 @@
 			<version>3.0.1</version>
 		</dependency>
 		<dependency>
-			<groupId>edu.uci.ics.asterix</groupId>
-			<artifactId>asterix-transactions</artifactId>
-			<version>0.0.6-SNAPSHOT</version>
-			<type>jar</type>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-storage-am-lsm-common</artifactId>
 		</dependency>
@@ -121,6 +114,30 @@
 			<version>4.8.1</version>
 			<scope>test</scope>
 		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-btree</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>
+				hyracks-storage-am-lsm-invertedindex
+			</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>com.fasterxml.jackson.core</groupId>
+			<artifactId>jackson-core</artifactId>
+			<version>2.2.0</version>
+		</dependency>
+		<dependency>
+			<groupId>org.codehaus.jackson</groupId>
+			<artifactId>jackson-mapper-asl</artifactId>
+			<version>1.9.12</version>
+		</dependency>
 	</dependencies>
 
 </project>
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixContextInfo.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixContextInfo.java
new file mode 100644
index 0000000..969aa65
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixContextInfo.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.asterix.common.api;
+
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+
+public class AsterixContextInfo implements IAsterixContextInfo {
+
+    public static AsterixContextInfo INSTANCE;
+
+    private final ICCApplicationContext appCtx;
+
+    public static void initialize(ICCApplicationContext ccAppCtx) {
+        if (INSTANCE == null) {
+            INSTANCE = new AsterixContextInfo(ccAppCtx);
+        }
+    }
+
+    private AsterixContextInfo(ICCApplicationContext ccAppCtx) {
+        this.appCtx = ccAppCtx;
+    }
+
+    @Override
+    public ICCApplicationContext getCCApplicationContext() {
+        return appCtx;
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
new file mode 100644
index 0000000..14975ff
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.common.api;
+
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+public class AsterixThreadExecutor implements Executor {
+    public final static AsterixThreadExecutor INSTANCE = new AsterixThreadExecutor();
+    private final Executor executor = Executors.newCachedThreadPool(AsterixThreadFactory.INSTANCE);
+
+    private AsterixThreadExecutor() {
+
+    }
+
+    @Override
+    public void execute(Runnable command) {
+        executor.execute(command);
+    }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadFactory.java
new file mode 100644
index 0000000..7e4735f
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.common.api;
+
+import java.util.concurrent.ThreadFactory;
+
+import edu.uci.ics.hyracks.api.lifecycle.LifeCycleComponentManager;
+
+public class AsterixThreadFactory implements ThreadFactory {
+
+    public final static AsterixThreadFactory INSTANCE = new AsterixThreadFactory();
+
+    private AsterixThreadFactory() {
+
+    }
+
+    @Override
+    public Thread newThread(Runnable r) {
+        Thread t;
+        if ((r instanceof Thread)) {
+            t = (Thread) r;
+        } else {
+            t = new Thread(r);
+        }
+        t.setUncaughtExceptionHandler(LifeCycleComponentManager.INSTANCE);
+        return t;
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
new file mode 100644
index 0000000..964b761
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -0,0 +1,52 @@
+package edu.uci.ics.asterix.common.api;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+
+public interface IAsterixAppRuntimeContext {
+
+    public ITransactionSubsystem getTransactionSubsystem();
+
+    public boolean isShuttingdown();
+
+    public ILSMIOOperationScheduler getLSMIOScheduler();
+
+    public int getMetaDataIODeviceId();
+
+    public ILSMMergePolicy getLSMMergePolicy();
+
+    public IBufferCache getBufferCache();
+
+    public IFileMapProvider getFileMapManager();
+
+    public ILocalResourceRepository getLocalResourceRepository();
+
+    public IIndexLifecycleManager getIndexLifecycleManager();
+
+    public ResourceIdFactory getResourceIdFactory();
+
+    public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID);
+
+    public void initialize() throws IOException, ACIDException, AsterixException;
+
+    public void setShuttingdown(boolean b);
+
+    public void deinitialize() throws HyracksDataException;
+
+    public double getBloomFilterFalsePositiveRate();
+
+    public IVirtualBufferCache getVirtualBufferCache(int datasetID);
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixContextInfo.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixContextInfo.java
new file mode 100644
index 0000000..1c43f2f
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixContextInfo.java
@@ -0,0 +1,15 @@
+package edu.uci.ics.asterix.common.api;
+
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+
+public interface IAsterixContextInfo {
+
+    /**
+     * Returns an instance of the implementation for ICCApplicationContext.
+     * 
+     * @return ICCApplicationContext implementation instance
+     */
+    public ICCApplicationContext getCCApplicationContext();
+
+ 
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IAsterixAppRuntimeContextProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixRuntimeComponentsProvider.java
similarity index 73%
copy from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IAsterixAppRuntimeContextProvider.java
copy to asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixRuntimeComponentsProvider.java
index d96d5c2..d7bc0f3 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IAsterixAppRuntimeContextProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixRuntimeComponentsProvider.java
@@ -1,36 +1,33 @@
-package edu.uci.ics.asterix.transaction.management.service.recovery;
+package edu.uci.ics.asterix.common.api;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 
-public interface IAsterixAppRuntimeContextProvider {
+public interface IAsterixRuntimeComponentsProvider {
 
     public IBufferCache getBufferCache();
 
     public IFileMapProvider getFileMapManager();
 
-    public TransactionSubsystem getTransactionSubsystem();
-
     public IIndexLifecycleManager getIndexLifecycleManager();
 
     public double getBloomFilterFalsePositiveRate();
 
     public ILSMMergePolicy getLSMMergePolicy();
 
-    public ILSMOperationTrackerFactory getLSMBTreeOperationTrackerFactory();
+    public ILSMOperationTrackerProvider getLSMBTreeOperationTrackerFactory();
 
-    public ILSMOperationTrackerFactory getLSMRTreeOperationTrackerFactory();
+    public ILSMOperationTrackerProvider getLSMRTreeOperationTrackerFactory();
 
-    public ILSMOperationTrackerFactory getLSMInvertedIndexOperationTrackerFactory();
+    public ILSMOperationTrackerProvider getLSMInvertedIndexOperationTrackerFactory();
 
     public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider();
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ILocalResourceMetadata.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/ILocalResourceMetadata.java
similarity index 70%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ILocalResourceMetadata.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/api/ILocalResourceMetadata.java
index d2aa164..9fdfe5f 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ILocalResourceMetadata.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/ILocalResourceMetadata.java
@@ -1,8 +1,8 @@
-package edu.uci.ics.asterix.transaction.management.resource;
+package edu.uci.ics.asterix.common.api;
 
 import java.io.Serializable;
 
-import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 
@@ -10,5 +10,6 @@
 
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) throws HyracksDataException;
-    
+
+    public int getDatasetID();
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
index cf38932..6dd2ced 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
@@ -8,7 +8,7 @@
     private static int EXTERNAL_WEBPORT_DEFAULT = 19001;
 
     private static final String EXTERNAL_LOGLEVEL_KEY = "log.level";
-    private static Level EXTERNAL_LOGLEVEL_DEFAULT = Level.INFO;
+    private static Level EXTERNAL_LOGLEVEL_DEFAULT = Level.WARNING;
 
     private static final String EXTERNAL_APISERVER_KEY = "api.port";
     private static int EXTERNAL_APISERVER_DEFAULT = 19101;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
index 6d47e78..6b6cded 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
@@ -24,5 +24,9 @@
     public Set<String> getNodeNames() {
         return accessor.getNodeNames();
     }
+    
+    public String getCoredumpPath(String nodeId){
+        return accessor.getCoredumpPath(nodeId);
+    }
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
index 7b2f2a6..d623ae5 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
@@ -16,6 +16,7 @@
 import javax.xml.bind.Unmarshaller;
 
 import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
+import edu.uci.ics.asterix.common.configuration.Coredump;
 import edu.uci.ics.asterix.common.configuration.Property;
 import edu.uci.ics.asterix.common.configuration.Store;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -26,6 +27,7 @@
     private final String metadataNodeName;
     private final Set<String> nodeNames;
     private final Map<String, String[]> stores;
+    private final Map<String, String> coredumpConfig;
     private final Map<String, Property> asterixConfigurationParams;
 
     public AsterixPropertiesAccessor() throws AsterixException {
@@ -64,6 +66,11 @@
         for (Property p : asterixConfiguration.getProperty()) {
             asterixConfigurationParams.put(p.getName(), p);
         }
+        coredumpConfig = new HashMap<String, String>();
+        for (Coredump cd : asterixConfiguration.getCoredump()) {
+            coredumpConfig.put(cd.getNcId(), cd.getCoredumpPath());
+        }
+
     }
 
     public String getMetadataNodeName() {
@@ -82,6 +89,10 @@
         return nodeNames;
     }
 
+    public String getCoredumpPath(String nodeId) {
+        return coredumpConfig.get(nodeId);
+    }
+
     public <T> T getProperty(String property, T defaultValue, IPropertyInterpreter<T> interpreter) {
         Property p = asterixConfigurationParams.get(property);
         if (p == null) {
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
index d34e4ac..753778a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
@@ -5,8 +5,8 @@
     private static final String STORAGE_BUFFERCACHE_PAGESIZE_KEY = "storage.buffercache.pagesize";
     private static int STORAGE_BUFFERCACHE_PAGESIZE_DEFAULT = (32 << 10); // 32KB
 
-    private static final String STORAGE_BUFFERCACHE_NUMPAGES_KEY = "storage.buffercache.numpages";
-    private static final int STORAGE_BUFFERCACHE_NUMPAGES_DEFAULT = 1024;
+    private static final String STORAGE_BUFFERCACHE_SIZE_KEY = "storage.buffercache.size";
+    private static final long STORAGE_BUFFERCACHE_SIZE_DEFAULT = (32 << 20); // 32 MB
 
     private static final String STORAGE_BUFFERCACHE_MAXOPENFILES_KEY = "storage.buffercache.maxopenfiles";
     private static int STORAGE_BUFFERCACHE_MAXOPENFILES_DEFAULT = Integer.MAX_VALUE;
@@ -15,10 +15,10 @@
     private static final int STORAGE_MEMORYCOMPONENT_PAGESIZE_DEFAULT = (32 << 10); // 32KB
 
     private static final String STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY = "storage.memorycomponent.numpages";
-    private static final int STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT = 4096; // ... so 128MB components
+    private static final int STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT = 1024; // ... so 32MB components
 
     private static final String STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY = "storage.memorycomponent.globalbudget";
-    private static final long STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT = (1 << 30); // 1GB
+    private static final long STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT = 536870912; // 512MB
 
     private static final String STORAGE_LSM_MERGETHRESHOLD_KEY = "storage.lsm.mergethreshold";
     private static int STORAGE_LSM_MERGETHRESHOLD_DEFAULT = 3;
@@ -35,9 +35,13 @@
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    public long getBufferCacheSize() {
+        return accessor.getProperty(STORAGE_BUFFERCACHE_SIZE_KEY, STORAGE_BUFFERCACHE_SIZE_DEFAULT,
+                PropertyInterpreters.getLongPropertyInterpreter());
+    }
+    
     public int getBufferCacheNumPages() {
-        return accessor.getProperty(STORAGE_BUFFERCACHE_NUMPAGES_KEY, STORAGE_BUFFERCACHE_NUMPAGES_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
+        return (int) (getBufferCacheSize() / getBufferCachePageSize());
     }
 
     public int getBufferCacheMaxOpenFiles() {
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
index d97e53b..0b6ea85 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
@@ -1,6 +1,9 @@
 package edu.uci.ics.asterix.common.config;
 
 public class AsterixTransactionProperties extends AbstractAsterixProperties {
+    private static final String TXN_LOG_DIRECTORY_KEY = "txn.log.directory";
+    private static final String TXN_LOG_DIRECTORY_DEFAULT = "asterix_logs/";
+    
     private static final String TXN_LOG_BUFFER_NUMPAGES_KEY = "txn.log.buffer.numpages";
     private static int TXN_LOG_BUFFER_NUMPAGES_DEFAULT = 8;
 
@@ -9,25 +12,46 @@
 
     private static final String TXN_LOG_PARTITIONSIZE_KEY = "txn.log.partitionsize";
     private static final long TXN_LOG_PARTITIONSIZE_DEFAULT = (2 << 30); // 2GB
+    
+    private static final String TXN_LOG_DISKSECTORSIZE_KEY = "txn.log.disksectorsize";
+    private static final int TXN_LOG_DISKSECTORSIZE_DEFAULT = 4096;
 
     private static final String TXN_LOG_GROUPCOMMITINTERVAL_KEY = "txn.log.groupcommitinterval";
-    private static int TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT = 200; // 200ms
+    private static int TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT = 10; // 0.1ms
 
     private static final String TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY = "txn.log.checkpoint.lsnthreshold";
     private static final int TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT = (64 << 20); // 64M
 
     private static final String TXN_LOG_CHECKPOINT_POLLFREQUENCY_KEY = "txn.log.checkpoint.pollfrequency";
     private static int TXN_LOG_CHECKPOINT_POLLFREQUENCY_DEFAULT = 120; // 120s
+    
+    private static final String TXN_LOG_CHECKPOINT_HISTORY_KEY = "txn.log.checkpoint.history";
+    private static int TXN_LOG_CHECKPOINT_HISTORY_DEFAULT = 0;
 
     private static final String TXN_LOCK_ESCALATIONTHRESHOLD_KEY = "txn.lock.escalationthreshold";
     private static int TXN_LOCK_ESCALATIONTHRESHOLD_DEFAULT = 1000;
 
     private static final String TXN_LOCK_SHRINKTIMER_KEY = "txn.lock.shrinktimer";
-    private static int TXN_LOCK_SHRINKTIMER_DEFAULT = 120000; // 2m
+    private static int TXN_LOCK_SHRINKTIMER_DEFAULT = 5000; // 5s
+    
+    private static final String TXN_LOCK_TIMEOUT_WAITTHRESHOLD_KEY = "txn.lock.timeout.waitthreshold";
+    private static final int TXN_LOCK_TIMEOUT_WAITTHRESHOLD_DEFAULT = 60000; // 60s
+    
+    private static final String TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_KEY = "txn.lock.timeout.sweepthreshold";
+    private static final int TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_DEFAULT = 10000; // 10s
 
     public AsterixTransactionProperties(AsterixPropertiesAccessor accessor) {
         super(accessor);
     }
+    
+    public String getLogDirectory() {
+        String logDirectory = accessor.getProperty(TXN_LOG_DIRECTORY_KEY, TXN_LOG_DIRECTORY_DEFAULT,
+                PropertyInterpreters.getStringPropertyInterpreter());
+        if (!logDirectory.endsWith("/")) {
+            logDirectory += "/";
+        }
+        return logDirectory;
+    }
 
     public int getLogBufferNumPages() {
         return accessor.getProperty(TXN_LOG_BUFFER_NUMPAGES_KEY, TXN_LOG_BUFFER_NUMPAGES_DEFAULT,
@@ -43,6 +67,11 @@
         return accessor.getProperty(TXN_LOG_PARTITIONSIZE_KEY, TXN_LOG_PARTITIONSIZE_DEFAULT,
                 PropertyInterpreters.getLongPropertyInterpreter());
     }
+    
+    public int getLogDiskSectorSize() {
+        return accessor.getProperty(TXN_LOG_DISKSECTORSIZE_KEY, TXN_LOG_DISKSECTORSIZE_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
 
     public int getGroupCommitInterval() {
         return accessor.getProperty(TXN_LOG_GROUPCOMMITINTERVAL_KEY, TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT,
@@ -59,6 +88,11 @@
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    public int getCheckpointHistory() {
+        return accessor.getProperty(TXN_LOG_CHECKPOINT_HISTORY_KEY, TXN_LOG_CHECKPOINT_HISTORY_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+    
     public int getEntityToDatasetLockEscalationThreshold() {
         return accessor.getProperty(TXN_LOCK_ESCALATIONTHRESHOLD_KEY, TXN_LOCK_ESCALATIONTHRESHOLD_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
@@ -68,5 +102,15 @@
         return accessor.getProperty(TXN_LOCK_SHRINKTIMER_KEY, TXN_LOCK_SHRINKTIMER_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
+    
+    public int getTimeoutWaitThreshold() {
+        return accessor.getProperty(TXN_LOCK_TIMEOUT_WAITTHRESHOLD_KEY, TXN_LOCK_TIMEOUT_WAITTHRESHOLD_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+    
+    public int getTimeoutSweepThreshold() {
+        return accessor.getProperty(TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_KEY, TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java
deleted file mode 100644
index fad0a38..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java
+++ /dev/null
@@ -1,96 +0,0 @@
-package edu.uci.ics.asterix.common.context;
-
-import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
-import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
-
-public class AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
-        ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider, ILSMOperationTrackerFactory,
-        ILSMIOOperationCallbackProvider {
-    private static final long serialVersionUID = 1L;
-
-    private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
-
-    public static final AsterixRuntimeComponentsProvider LSMBTREE_PROVIDER = new AsterixRuntimeComponentsProvider(
-            LSMBTreeIOOperationCallbackFactory.INSTANCE);
-    public static final AsterixRuntimeComponentsProvider LSMRTREE_PROVIDER = new AsterixRuntimeComponentsProvider(
-            LSMRTreeIOOperationCallbackFactory.INSTANCE);
-    public static final AsterixRuntimeComponentsProvider LSMINVERTEDINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(
-            LSMInvertedIndexIOOperationCallbackFactory.INSTANCE);
-    public static final AsterixRuntimeComponentsProvider NOINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(null);
-
-    private AsterixRuntimeComponentsProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
-        this.ioOpCallbackFactory = ioOpCallbackFactory;
-    }
-
-    @Override
-    public ILSMOperationTracker createOperationTracker(ILSMIndex index) {
-        return new IndexOperationTracker(index, ioOpCallbackFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback getIOOperationCallback(ILSMIndex index) {
-        return ((IndexOperationTracker) index.getOperationTracker()).getIOOperationCallback();
-    }
-
-    @Override
-    public ILSMIOOperationScheduler getIOScheduler(IHyracksTaskContext ctx) {
-        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getLSMIOScheduler();
-    }
-
-    @Override
-    public ILSMMergePolicy getMergePolicy(IHyracksTaskContext ctx) {
-        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getLSMMergePolicy();
-    }
-
-    @Override
-    public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
-        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getBufferCache();
-    }
-
-    @Override
-    public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
-        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getFileMapManager();
-    }
-
-    @Override
-    public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
-        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getLocalResourceRepository();
-    }
-
-    @Override
-    public IIndexLifecycleManager getLifecycleManager(IHyracksTaskContext ctx) {
-        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getIndexLifecycleManager();
-    }
-
-    @Override
-    public ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
-        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getResourceIdFactory();
-    }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
new file mode 100644
index 0000000..c046e42
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.asterix.common.context;
+
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
+
+public class AsterixVirtualBufferCacheProvider implements IVirtualBufferCacheProvider {
+
+    private static final long serialVersionUID = 1L;
+    private final int datasetID;
+
+    public AsterixVirtualBufferCacheProvider(int datasetID) {
+        this.datasetID = datasetID;
+    }
+
+    @Override
+    public IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx) {
+        return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getVirtualBufferCache(datasetID);
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
new file mode 100644
index 0000000..6313a09
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
@@ -0,0 +1,66 @@
+package edu.uci.ics.asterix.common.context;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+
+public class BaseOperationTracker implements ILSMOperationTracker {
+
+    protected final ILSMIOOperationCallback ioOpCallback;
+    protected long lastLSN;
+    protected long firstLSN;
+
+    public BaseOperationTracker(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+        this.ioOpCallback = ioOpCallbackFactory == null ? NoOpIOOperationCallback.INSTANCE : ioOpCallbackFactory
+                .createIOOperationCallback(this);
+        resetLSNs();
+    }
+
+    public ILSMIOOperationCallback getIOOperationCallback() {
+        return ioOpCallback;
+    }
+
+    public long getLastLSN() {
+        return lastLSN;
+    }
+
+    public long getFirstLSN() {
+        return firstLSN;
+    }
+
+    public void updateLastLSN(long lastLSN) {
+        if (firstLSN == -1) {
+            firstLSN = lastLSN;
+        }
+        this.lastLSN = Math.max(this.lastLSN, lastLSN);
+    }
+
+    public void resetLSNs() {
+        lastLSN = -1;
+        firstLSN = -1;
+    }
+
+    @Override
+    public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+    }
+
+    @Override
+    public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+    }
+
+    @Override
+    public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+    }
+
+    public void exclusiveJobCommitted() throws HyracksDataException {
+    }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
index 74b0aa6..2daed08 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.asterix.common.context;
 
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
@@ -26,9 +27,9 @@
 public class ConstantMergePolicy implements ILSMMergePolicy {
 
     private final int threshold;
-    private final AsterixAppRuntimeContext ctx;
+    private final IAsterixAppRuntimeContext ctx;
 
-    public ConstantMergePolicy(int threshold, AsterixAppRuntimeContext ctx) {
+    public ConstantMergePolicy(int threshold, IAsterixAppRuntimeContext ctx) {
         this.threshold = threshold;
         this.ctx = ctx;
     }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
new file mode 100644
index 0000000..434835b
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
@@ -0,0 +1,345 @@
+package edu.uci.ics.asterix.common.context;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.LocalResource;
+
+public class DatasetLifecycleManager implements IIndexLifecycleManager, ILifeCycleComponent {
+    private final AsterixStorageProperties storageProperties;
+    private final Map<Integer, MultitenantVirtualBufferCache> datasetVirtualBufferCaches;
+    private final Map<Integer, ILSMOperationTracker> datasetOpTrackers;
+    private final Map<Integer, DatasetInfo> datasetInfos;
+    private final ILocalResourceRepository resourceRepository;
+    private final long capacity;
+    private long used;
+
+    public DatasetLifecycleManager(AsterixStorageProperties storageProperties,
+            ILocalResourceRepository resourceRepository) {
+        this.storageProperties = storageProperties;
+        this.resourceRepository = resourceRepository;
+        datasetVirtualBufferCaches = new HashMap<Integer, MultitenantVirtualBufferCache>();
+        datasetOpTrackers = new HashMap<Integer, ILSMOperationTracker>();
+        datasetInfos = new HashMap<Integer, DatasetInfo>();
+        capacity = storageProperties.getMemoryComponentGlobalBudget();
+        used = 0;
+    }
+
+    @Override
+    public synchronized IIndex getIndex(long resourceID) throws HyracksDataException {
+        DatasetInfo dsInfo = datasetInfos.get(getDIDfromRID(resourceID));
+        if (dsInfo == null) {
+            return null;
+        }
+        IndexInfo iInfo = dsInfo.indexes.get(resourceID);
+        if (iInfo == null) {
+            return null;
+        }
+        return iInfo.index;
+    }
+
+    @Override
+    public synchronized void register(long resourceID, IIndex index) throws HyracksDataException {
+        int did = getDIDfromRID(resourceID);
+        DatasetInfo dsInfo = datasetInfos.get(did);
+        if (dsInfo == null) {
+            dsInfo = new DatasetInfo(did);
+        } else if (dsInfo.indexes.containsKey(resourceID)) {
+            throw new HyracksDataException("Index with resource ID " + resourceID + " already exists.");
+        }
+        datasetInfos.put(did, dsInfo);
+        dsInfo.indexes.put(resourceID, new IndexInfo((ILSMIndex) index));
+    }
+
+    private int getDIDfromRID(long resourceID) throws HyracksDataException {
+        LocalResource lr = resourceRepository.getResourceById(resourceID);
+        if (lr == null) {
+            return -1;
+        }
+        return ((ILocalResourceMetadata) lr.getResourceObject()).getDatasetID();
+    }
+
+    @Override
+    public synchronized void unregister(long resourceID) throws HyracksDataException {
+        int did = getDIDfromRID(resourceID);
+        DatasetInfo dsInfo = datasetInfos.get(did);
+        IndexInfo iInfo = dsInfo.indexes.remove(resourceID);
+        if (dsInfo == null || iInfo == null) {
+            throw new HyracksDataException("Index with resource ID " + resourceID + " does not exist.");
+        }
+
+        if (iInfo.referenceCount != 0) {
+            dsInfo.indexes.put(resourceID, iInfo);
+            throw new HyracksDataException("Cannot remove index while it is open.");
+        }
+
+        if (iInfo.isOpen) {
+            iInfo.index.deactivate(true);
+        }
+
+        if (dsInfo.referenceCount == 0 && dsInfo.isOpen && dsInfo.indexes.isEmpty()) {
+            IVirtualBufferCache vbc = getVirtualBufferCache(did);
+            assert vbc != null;
+            used -= (vbc.getNumPages() * vbc.getPageSize());
+            datasetInfos.remove(did);
+        }
+
+    }
+
+    @Override
+    public synchronized void open(long resourceID) throws HyracksDataException {
+        int did = getDIDfromRID(resourceID);
+        DatasetInfo dsInfo = datasetInfos.get(did);
+        if (dsInfo == null) {
+            throw new HyracksDataException("Failed to open index with resource ID " + resourceID
+                    + " since it does not exist.");
+        }
+
+        IndexInfo iInfo = dsInfo.indexes.get(resourceID);
+        if (iInfo == null) {
+            throw new HyracksDataException("Failed to open index with resource ID " + resourceID
+                    + " since it does not exist.");
+        }
+
+        if (!dsInfo.isOpen) {
+            IVirtualBufferCache vbc = getVirtualBufferCache(did);
+            assert vbc != null;
+            long additionalSize = vbc.getNumPages() * vbc.getPageSize();
+            while (used + additionalSize > capacity) {
+                if (!evictCandidateDataset()) {
+                    throw new HyracksDataException("Cannot activate index since memory budget would be exceeded.");
+                }
+            }
+            used += additionalSize;
+        }
+
+        dsInfo.isOpen = true;
+        dsInfo.touch();
+        if (!iInfo.isOpen) {
+            iInfo.index.activate();
+            iInfo.isOpen = true;
+        }
+        iInfo.touch();
+    }
+
+    private boolean evictCandidateDataset() throws HyracksDataException {
+        // Why min()? As a heuristic for eviction, we will take an open index (an index consuming memory) 
+        // that is not being used (refcount == 0) and has been least recently used. The sort order defined 
+        // for IndexInfo maintains this. See IndexInfo.compareTo().
+        DatasetInfo dsInfo = Collections.min(datasetInfos.values());
+        if (dsInfo.referenceCount == 0 && dsInfo.isOpen) {
+            for (IndexInfo iInfo : dsInfo.indexes.values()) {
+                if (iInfo.isOpen) {
+                    iInfo.index.deactivate(true);
+                    iInfo.isOpen = false;
+                }
+                assert iInfo.referenceCount == 0;
+            }
+
+            IVirtualBufferCache vbc = getVirtualBufferCache(dsInfo.datasetID);
+            used -= vbc.getNumPages() * vbc.getPageSize();
+            dsInfo.isOpen = false;
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public synchronized void close(long resourceID) throws HyracksDataException {
+        int did = getDIDfromRID(resourceID);
+        DatasetInfo dsInfo = datasetInfos.get(did);
+        if (dsInfo == null) {
+            throw new HyracksDataException("No index found with resourceID " + resourceID);
+        }
+        IndexInfo iInfo = dsInfo.indexes.get(resourceID);
+        if (iInfo == null) {
+            throw new HyracksDataException("No index found with resourceID " + resourceID);
+        }
+        iInfo.untouch();
+        dsInfo.untouch();
+    }
+
+    @Override
+    public synchronized List<IIndex> getOpenIndexes() {
+        List<IIndex> openIndexes = new ArrayList<IIndex>();
+        for (DatasetInfo dsInfo : datasetInfos.values()) {
+            for (IndexInfo iInfo : dsInfo.indexes.values()) {
+                if (iInfo.isOpen) {
+                    openIndexes.add(iInfo.index);
+                }
+            }
+        }
+        return openIndexes;
+    }
+
+    public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
+        synchronized (datasetVirtualBufferCaches) {
+            MultitenantVirtualBufferCache vbc = datasetVirtualBufferCaches.get(datasetID);
+            if (vbc == null) {
+                vbc = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(),
+                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages()));
+                datasetVirtualBufferCaches.put(datasetID, vbc);
+            }
+            return vbc;
+        }
+    }
+
+    public ILSMOperationTracker getOperationTracker(int datasetID) {
+        synchronized (datasetOpTrackers) {
+            ILSMOperationTracker opTracker = datasetOpTrackers.get(datasetID);
+            if (opTracker == null) {
+                opTracker = new PrimaryIndexOperationTracker(this, datasetID,
+                        LSMBTreeIOOperationCallbackFactory.INSTANCE);
+                datasetOpTrackers.put(datasetID, opTracker);
+            }
+
+            return opTracker;
+        }
+    }
+
+    public synchronized Set<ILSMIndex> getDatasetIndexes(int datasetID) throws HyracksDataException {
+        DatasetInfo dsInfo = datasetInfos.get(datasetID);
+        if (dsInfo == null) {
+            throw new HyracksDataException("No dataset found with datasetID " + datasetID);
+        }
+        Set<ILSMIndex> datasetIndexes = new HashSet<ILSMIndex>();
+        for (IndexInfo iInfo : dsInfo.indexes.values()) {
+            datasetIndexes.add(iInfo.index);
+        }
+        return datasetIndexes;
+    }
+
+    private static abstract class Info {
+        protected int referenceCount;
+        protected boolean isOpen;
+
+        public Info() {
+            referenceCount = 0;
+            isOpen = false;
+        }
+
+        public void touch() {
+            ++referenceCount;
+        }
+
+        public void untouch() {
+            --referenceCount;
+        }
+    }
+
+    private static class IndexInfo extends Info {
+        private ILSMIndex index;
+
+        public IndexInfo(ILSMIndex index) {
+            this.index = index;
+        }
+    }
+
+    private static class DatasetInfo extends Info implements Comparable<DatasetInfo> {
+        private final Map<Long, IndexInfo> indexes;
+        private final int datasetID;
+        private long lastAccess;
+
+        public DatasetInfo(int datasetID) {
+            this.indexes = new HashMap<Long, IndexInfo>();
+            this.lastAccess = -1;
+            this.datasetID = datasetID;
+        }
+
+        public void touch() {
+            super.touch();
+            lastAccess = System.currentTimeMillis();
+        }
+
+        public void untouch() {
+            super.untouch();
+            lastAccess = System.currentTimeMillis();
+        }
+
+        @Override
+        public int compareTo(DatasetInfo i) {
+            // sort by (isOpen, referenceCount, lastAccess) ascending, where true < false
+            //
+            // Example sort order:
+            // -------------------
+            // (F, 0, 70)       <-- largest
+            // (F, 0, 60)
+            // (T, 10, 80)
+            // (T, 10, 70)
+            // (T, 9, 90)
+            // (T, 0, 100)      <-- smallest
+            if (isOpen && !i.isOpen) {
+                return -1;
+            } else if (!isOpen && i.isOpen) {
+                return 1;
+            } else {
+                if (referenceCount < i.referenceCount) {
+                    return -1;
+                } else if (referenceCount > i.referenceCount) {
+                    return 1;
+                } else {
+                    if (lastAccess < i.lastAccess) {
+                        return -1;
+                    } else if (lastAccess > i.lastAccess) {
+                        return 1;
+                    } else {
+                        return 0;
+                    }
+                }
+            }
+
+        }
+
+        public String toString() {
+            return "DatasetID: " + datasetID + ", isOpen: " + isOpen + ", refCount: " + referenceCount
+                    + ", lastAccess: " + lastAccess + "}";
+        }
+    }
+
+    @Override
+    public void start() {
+    }
+
+    @Override
+    public void stop(boolean dumpState, OutputStream outputStream) throws IOException {
+        StringBuilder sb = new StringBuilder();
+
+        sb.append(String.format("Memory budget = %d\n", capacity));
+        sb.append(String.format("Memory used = %d\n", used));
+
+        String headerFormat = "%-20s %-10s %-20s %-20s\n";
+        String dsFormat = "%-20d %-10b %-20d %-20s %-20s\n";
+        String idxFormat = "\t%-20d %-10b %-20d %-20s\n";
+        sb.append(String.format(headerFormat, "DatasetID", "Open", "Reference Count", "Last Access"));
+        for (DatasetInfo dsInfo : datasetInfos.values()) {
+            sb.append(String
+                    .format(dsFormat, dsInfo.datasetID, dsInfo.isOpen, dsInfo.referenceCount, dsInfo.lastAccess));
+            for (Map.Entry<Long, IndexInfo> entry : dsInfo.indexes.entrySet()) {
+                IndexInfo iInfo = entry.getValue();
+                sb.append(String.format(idxFormat, entry.getKey(), iInfo.isOpen, iInfo.referenceCount, iInfo.index));
+            }
+        }
+        outputStream.write(sb.toString().getBytes());
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionSubsystemProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ITransactionSubsystemProvider.java
similarity index 79%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionSubsystemProvider.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ITransactionSubsystemProvider.java
index 857d8ae..a95890b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionSubsystemProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ITransactionSubsystemProvider.java
@@ -13,12 +13,13 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.service.transaction;
+package edu.uci.ics.asterix.common.context;
 
 import java.io.Serializable;
 
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 public interface ITransactionSubsystemProvider extends Serializable{
-    public TransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx);
+    public ITransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx);
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
new file mode 100644
index 0000000..b924024
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -0,0 +1,116 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.common.context;
+
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+
+public class PrimaryIndexOperationTracker extends BaseOperationTracker {
+
+    private final DatasetLifecycleManager datasetLifecycleManager;
+    private final IVirtualBufferCache datasetBufferCache;
+    private final int datasetID;
+    // Number of active operations on a ILSMIndex instance.
+    private AtomicInteger numActiveOperations;
+
+    public PrimaryIndexOperationTracker(DatasetLifecycleManager datasetLifecycleManager, int datasetID,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+        super(ioOpCallbackFactory);
+        this.datasetLifecycleManager = datasetLifecycleManager;
+        this.numActiveOperations = new AtomicInteger(0);
+        this.datasetID = datasetID;
+        datasetBufferCache = datasetLifecycleManager.getVirtualBufferCache(datasetID);
+    }
+
+    @Override
+    public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        numActiveOperations.incrementAndGet();
+
+        // Increment transactor-local active operations count.
+        AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
+        if (opCallback != null) {
+            opCallback.incrementLocalNumActiveOperations();
+        }
+    }
+
+    @Override
+    public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        // Searches are immediately considered complete, because they should not prevent the execution of flushes.
+        if ((searchCallback != null && searchCallback != NoOpOperationCallback.INSTANCE)
+                || opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+            completeOperation(index, opType, searchCallback, modificationCallback);
+        }
+    }
+
+    @Override
+    public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        int nActiveOps = numActiveOperations.decrementAndGet();
+        // Decrement transactor-local active operations count.
+        AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
+        if (opCallback != null) {
+            opCallback.decrementLocalNumActiveOperations();
+        }
+        if (opType != LSMOperationType.FLUSH) {
+            flushIfFull(nActiveOps);
+        }
+    }
+
+    private void flushIfFull(int nActiveOps) throws HyracksDataException {
+        // If we need a flush, and this is the last completing operation, then schedule the flush.
+        if (datasetBufferCache.isFull() && nActiveOps == 0) {
+            Set<ILSMIndex> indexes = datasetLifecycleManager.getDatasetIndexes(datasetID);
+            for (ILSMIndex lsmIndex : indexes) {
+                ILSMIndexAccessor accessor = (ILSMIndexAccessor) lsmIndex.createAccessor(
+                        NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+                accessor.scheduleFlush(((BaseOperationTracker) lsmIndex.getOperationTracker()).getIOOperationCallback());
+            }
+
+        }
+    }
+
+    public void exclusiveJobCommitted() throws HyracksDataException {
+        numActiveOperations.set(0);
+        flushIfFull(0);
+    }
+
+    private AbstractOperationCallback getOperationCallback(ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) {
+
+        if (modificationCallback == NoOpOperationCallback.INSTANCE || modificationCallback == null) {
+            return null;
+        }
+        if (searchCallback != null && searchCallback != NoOpOperationCallback.INSTANCE) {
+            return (AbstractOperationCallback) searchCallback;
+        } else {
+            return (AbstractOperationCallback) modificationCallback;
+        }
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/TransactionSubsystemProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/TransactionSubsystemProvider.java
index 85e5375..33f20f7 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/TransactionSubsystemProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/TransactionSubsystemProvider.java
@@ -15,8 +15,8 @@
 
 package edu.uci.ics.asterix.common.context;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 /**
@@ -26,8 +26,8 @@
  */
 public class TransactionSubsystemProvider implements ITransactionSubsystemProvider {
     @Override
-    public TransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx) {
-        AsterixAppRuntimeContext runtimeCtx = (AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+    public ITransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx) {
+        IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
                 .getApplicationObject();
         return runtimeCtx.getTransactionSubsystem();
     }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
new file mode 100644
index 0000000..93a86e4
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -0,0 +1,71 @@
+package edu.uci.ics.asterix.common.dataflow;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
+
+public class AsterixLSMInsertDeleteOperatorNodePushable extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
+
+    private final boolean isPrimary;
+
+    public AsterixLSMInsertDeleteOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider, IndexOperation op,
+            boolean isPrimary) {
+        super(opDesc, ctx, partition, fieldPermutation, recordDescProvider, op);
+        this.isPrimary = isPrimary;
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        boolean first = true;
+        accessor.reset(buffer);
+        ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
+        int tupleCount = accessor.getTupleCount();
+        try {
+            for (int i = 0; i < tupleCount; i++) {
+                if (tupleFilter != null) {
+                    frameTuple.reset(accessor, i);
+                    if (!tupleFilter.accept(frameTuple)) {
+                        lsmAccessor.noOp();
+                        continue;
+                    }
+                }
+                tuple.reset(accessor, i);
+                switch (op) {
+                    case INSERT:
+                        if (first && isPrimary) {
+                            lsmAccessor.insert(tuple);
+                            first = false;
+                        } else {
+                            lsmAccessor.forceInsert(tuple);
+                        }
+                        break;
+                    case DELETE:
+                        if (first && isPrimary) {
+                            lsmAccessor.delete(tuple);
+                            first = false;
+                        } else {
+                            lsmAccessor.forceDelete(tuple);
+                        }
+                        break;
+                    default: {
+                        throw new HyracksDataException("Unsupported operation " + op
+                                + " in tree index InsertDelete operator");
+                    }
+                }
+            }
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+        System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
+        FrameUtils.flushFrame(writeBuffer, writer);
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor.java
new file mode 100644
index 0000000..15ad465
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor.java
@@ -0,0 +1,42 @@
+package edu.uci.ics.asterix.common.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexInsertUpdateDeleteOperator;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public class AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor extends LSMInvertedIndexInsertUpdateDeleteOperator {
+
+    private static final long serialVersionUID = 1L;
+
+    public AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor(IOperatorDescriptorRegistry spec,
+            RecordDescriptor recDesc, IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+            IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
+            IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
+            int[] fieldPermutation, IndexOperation op, IIndexDataflowHelperFactory dataflowHelperFactory,
+            ITupleFilterFactory tupleFilterFactory, IModificationOperationCallbackFactory modificationOpCallbackFactory) {
+        super(spec, recDesc, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
+                tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
+                fieldPermutation, op, dataflowHelperFactory, tupleFilterFactory, modificationOpCallbackFactory);
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new AsterixLSMInsertDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
+                recordDescProvider, op, false);
+    }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java
new file mode 100644
index 0000000..b1bd6ce
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMTreeInsertDeleteOperatorDescriptor.java
@@ -0,0 +1,45 @@
+package edu.uci.ics.asterix.common.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public class AsterixLSMTreeInsertDeleteOperatorDescriptor extends LSMTreeIndexInsertUpdateDeleteOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private final boolean isPrimary;
+
+    public AsterixLSMTreeInsertDeleteOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
+            IndexOperation op, IIndexDataflowHelperFactory dataflowHelperFactory,
+            ITupleFilterFactory tupleFilterFactory,
+            IModificationOperationCallbackFactory modificationOpCallbackProvider, boolean isPrimary) {
+        super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, bloomFilterKeyFields, fieldPermutation, op, dataflowHelperFactory,
+                tupleFilterFactory, modificationOpCallbackProvider);
+        this.isPrimary = isPrimary;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new AsterixLSMInsertDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
+                recordDescProvider, op, isPrimary);
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java
index 401af28..a057f91 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java
@@ -19,10 +19,16 @@
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
 /**
- * Provides methods for obtaining the IIndexRegistryProvider, IStorageManager and
+ * Provides methods for obtaining the IIndexLifecycleManagerProvider, IStorageManagerInterface and
  * ICCApplicationContext implementation.
  */
 public interface IAsterixApplicationContextInfo {
+
+    /**
+     * Returns an instance of the implementation for IIndexLifecycleManagerProvider.
+     * 
+     * @return IIndexLifecycleManagerProvider implementation instance
+     */
     public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider();
 
     /**
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/exception/ACIDException.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/exceptions/ACIDException.java
similarity index 74%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/exception/ACIDException.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/exceptions/ACIDException.java
index 2bc9935..eb00d10 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/exception/ACIDException.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/exceptions/ACIDException.java
@@ -12,14 +12,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.exception;
+package edu.uci.ics.asterix.common.exceptions;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 
 /**
  * Represents an exception related to an unexpected behavior that prevents the
  * system from supporting ACID guarantees. The exception contains the
- * TransactionContext but it may not always be set. For example, an
+ * ITransactionContext but it may not always be set. For example, an
  * ACIDException encountered during crash recovery shall not have a transaction
  * context as recovery does not happen as part of a transaction.
  */
@@ -29,22 +29,22 @@
 	 * 
 	 */
     private static final long serialVersionUID = -8855848112541877323L;
-    private TransactionContext txnContext;
+    private ITransactionContext txnContext;
 
-    public TransactionContext getTxnContext() {
+    public ITransactionContext getTxnContext() {
         return txnContext;
     }
 
-    public void setTxnContext(TransactionContext txnContext) {
+    public void setTxnContext(ITransactionContext txnContext) {
         this.txnContext = txnContext;
     }
 
-    public ACIDException(TransactionContext txnContext, String message) {
+    public ACIDException(ITransactionContext txnContext, String message) {
         super(message);
         this.txnContext = txnContext;
     }
 
-    public ACIDException(TransactionContext txnContext, String message, Throwable cause) {
+    public ACIDException(ITransactionContext txnContext, String message, Throwable cause) {
         super(message, cause);
         this.txnContext = txnContext;
     }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionSignature.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionSignature.java
index 563b2cf..7520b1f 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionSignature.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionSignature.java
@@ -30,7 +30,7 @@
     }
 
     public String toString() {
-        return rep;
+        return namespace + "." + name + "@" + arity;
     }
 
     @Override
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
similarity index 91%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/AbstractLSMIOOperationCallback.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
index 74f39ad..c9c01e8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
@@ -13,11 +13,11 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.ioopcallbacks;
+package edu.uci.ics.asterix.common.ioopcallbacks;
 
 import java.util.List;
 
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
@@ -29,9 +29,9 @@
 
 public abstract class AbstractLSMIOOperationCallback implements ILSMIOOperationCallback {
 
-    protected final IndexOperationTracker opTracker;
+    protected final BaseOperationTracker opTracker;
 
-    public AbstractLSMIOOperationCallback(IndexOperationTracker opTracker) {
+    public AbstractLSMIOOperationCallback(BaseOperationTracker opTracker) {
         this.opTracker = opTracker;
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
similarity index 89%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallback.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
index 382a6d0..1fe77a0 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
@@ -13,11 +13,11 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.ioopcallbacks;
+package edu.uci.ics.asterix.common.ioopcallbacks;
 
 import java.util.List;
 
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeImmutableComponent;
@@ -25,7 +25,7 @@
 
 public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
 
-    public LSMBTreeIOOperationCallback(IndexOperationTracker opTracker) {
+    public LSMBTreeIOOperationCallback(BaseOperationTracker opTracker) {
         super(opTracker);
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
similarity index 83%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
index a51da07..7524ddb 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
@@ -13,23 +13,23 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.ioopcallbacks;
+package edu.uci.ics.asterix.common.ioopcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 
 public class LSMBTreeIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
 
     private static final long serialVersionUID = 1L;
-    
+
     public static LSMBTreeIOOperationCallbackFactory INSTANCE = new LSMBTreeIOOperationCallbackFactory();
-    
+
     private LSMBTreeIOOperationCallbackFactory() {
     }
 
     @Override
     public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
-        return new LSMBTreeIOOperationCallback((IndexOperationTracker) syncObj);
+        return new LSMBTreeIOOperationCallback((BaseOperationTracker) syncObj);
     }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
similarity index 89%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
index 0782c67..fb30742 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
@@ -13,18 +13,18 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.ioopcallbacks;
+package edu.uci.ics.asterix.common.ioopcallbacks;
 
 import java.util.List;
 
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexImmutableComponent;
 
 public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
 
-    public LSMInvertedIndexIOOperationCallback(IndexOperationTracker opTracker) {
+    public LSMInvertedIndexIOOperationCallback(BaseOperationTracker opTracker) {
         super(opTracker);
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
similarity index 82%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
index 790c60c..5e5b0ed 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
@@ -13,9 +13,9 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.ioopcallbacks;
+package edu.uci.ics.asterix.common.ioopcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 
@@ -24,12 +24,12 @@
     private static final long serialVersionUID = 1L;
 
     public static LSMInvertedIndexIOOperationCallbackFactory INSTANCE = new LSMInvertedIndexIOOperationCallbackFactory();
-    
+
     private LSMInvertedIndexIOOperationCallbackFactory() {
     }
-    
+
     @Override
     public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
-        return new LSMInvertedIndexIOOperationCallback((IndexOperationTracker) syncObj);
+        return new LSMInvertedIndexIOOperationCallback((BaseOperationTracker) syncObj);
     }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
similarity index 89%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallback.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
index b2a59b4..324cccb 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
@@ -13,18 +13,18 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.ioopcallbacks;
+package edu.uci.ics.asterix.common.ioopcallbacks;
 
 import java.util.List;
 
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeImmutableComponent;
 
 public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
 
-    public LSMRTreeIOOperationCallback(IndexOperationTracker opTracker) {
+    public LSMRTreeIOOperationCallback(BaseOperationTracker opTracker) {
         super(opTracker);
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
similarity index 83%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
index 4b47a95..748d93a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
@@ -13,23 +13,23 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.ioopcallbacks;
+package edu.uci.ics.asterix.common.ioopcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 
 public class LSMRTreeIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
 
     private static final long serialVersionUID = 1L;
-    
+
     public static LSMRTreeIOOperationCallbackFactory INSTANCE = new LSMRTreeIOOperationCallbackFactory();
-    
+
     private LSMRTreeIOOperationCallbackFactory() {
     }
 
     @Override
     public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
-        return new LSMRTreeIOOperationCallback((IndexOperationTracker) syncObj);
+        return new LSMRTreeIOOperationCallback((BaseOperationTracker) syncObj);
     }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
similarity index 66%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallback.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
index fdbb707..5e53060 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
@@ -13,52 +13,48 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.opcallbacks;
+package edu.uci.ics.asterix.common.transactions;
 
-import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
 
 public abstract class AbstractOperationCallback {
-    
+
     private final static long SEED = 0L;
-    
+
     protected final DatasetId datasetId;
     protected final int[] primaryKeyFields;
+    protected final ITransactionContext txnCtx;
     protected final ILockManager lockManager;
-    protected final TransactionContext txnCtx;
-    protected int transactorLocalNumActiveOperations = 0;
+    protected final AtomicInteger transactorLocalNumActiveOperations;
     protected final long[] longHashes;
 
-    public AbstractOperationCallback(int datasetId, int[] primaryKeyFields,
-            TransactionContext txnCtx, ILockManager lockManager) {
+    public AbstractOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
+            ILockManager lockManager) {
         this.datasetId = new DatasetId(datasetId);
         this.primaryKeyFields = primaryKeyFields;
         this.txnCtx = txnCtx;
         this.lockManager = lockManager;
-        this.longHashes= new long[2];
+        this.transactorLocalNumActiveOperations = new AtomicInteger(0);
+        this.longHashes = new long[2];
     }
 
     public int computePrimaryKeyHashValue(ITupleReference tuple, int[] primaryKeyFields) {
         MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
-        return Math.abs((int) longHashes[0]); 
-    }
-
-    public TransactionContext getTransactionContext() {
-        return txnCtx;
+        return Math.abs((int) longHashes[0]);
     }
 
     public int getLocalNumActiveOperations() {
-        return transactorLocalNumActiveOperations;
+        return transactorLocalNumActiveOperations.get();
     }
 
     public void incrementLocalNumActiveOperations() {
-        transactorLocalNumActiveOperations++;
+        transactorLocalNumActiveOperations.incrementAndGet();
     }
 
     public void decrementLocalNumActiveOperations() {
-        transactorLocalNumActiveOperations--;
+        transactorLocalNumActiveOperations.decrementAndGet();
     }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallbackFactory.java
similarity index 85%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallbackFactory.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallbackFactory.java
index 386dce5..5c0d5f1 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallbackFactory.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallbackFactory.java
@@ -13,12 +13,12 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.opcallbacks;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.io.Serializable;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.common.transactions.JobId;
 
 public abstract class AbstractOperationCallbackFactory implements Serializable {
     private static final long serialVersionUID = 1L;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java
new file mode 100644
index 0000000..d70288f
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java
@@ -0,0 +1,143 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.common.transactions;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
+ * transaction support. This class is not "thread-safe". For performance
+ * concerns, it is required for multiple writers to be able to write to the
+ * buffer concurrently and that a writer is never blocked by another writer. The
+ * users of this class must ensure that two concurrent writers get to write in
+ * exclusive areas in the buffer. A reader and writer may or may not conflict
+ * with each other. For example, reading of logs during roll back of a
+ * transaction t1 does not conflict with writing of logs by another transaction
+ * t2 as they are concerned with exclusive areas of the buffer. On the contrary,
+ * a flushing the buffer to disk conflicts with a reader reading the buffer.
+ * Appropriate locks are taken on the Buffer in the application logic and not
+ * directly imposed by synchronized methods.
+ */
+
+public class Buffer implements IBuffer {
+
+    ByteBuffer buffer;
+
+    public Buffer(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    protected Buffer() {
+    }
+
+    public void setBuffer(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    @Override
+    public byte getByte(int offset) {
+        return buffer.get(offset);
+    }
+
+    @Override
+    public byte getByte() {
+        return buffer.get();
+    }
+
+    @Override
+    public void getBytes(byte[] bytes, int offset, int size) {
+        System.arraycopy(buffer.array(), offset, bytes, 0, size);
+    }
+
+    @Override
+    public int getSize() {
+        return buffer.capacity();
+    }
+
+    @Override
+    public int readInt() {
+        return buffer.getInt();
+    }
+
+    @Override
+    public int readInt(int offset) {
+        return buffer.getInt(offset);
+    }
+
+    @Override
+    public long readLong(int offset) {
+        return buffer.getLong(offset);
+    }
+
+    @Override
+    public void put(byte b) {
+        buffer.put(b);
+    }
+
+    @Override
+    public void put(int offset, byte b) {
+        buffer.put(offset, b);
+    }
+
+    @Override
+    public void put(byte[] bytes, int start, int length) {
+        buffer.put(bytes, start, length);
+
+    }
+
+    @Override
+    public void put(byte[] bytes) {
+        buffer.put(bytes);
+    }
+
+    @Override
+    public void writeInt(int value) {
+        buffer.putInt(value);
+    }
+
+    @Override
+    public void writeInt(int offset, int value) {
+        buffer.putInt(offset, value);
+
+    }
+
+    @Override
+    public void writeLong(long value) {
+        buffer.putLong(value);
+    }
+
+    @Override
+    public void writeLong(int offset, long value) {
+        buffer.putLong(offset, value);
+
+    }
+
+    @Override
+    public byte[] getArray() {
+        return buffer.array();
+    }
+
+    @Override
+    public void erase() {
+        Arrays.fill(buffer.array(), (byte) 0);
+    }
+
+    @Override
+    public ByteBuffer getByteBuffer() {
+        return buffer;
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/DatasetId.java
similarity index 80%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetId.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/DatasetId.java
index 9aded2a..9f964f1 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetId.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/DatasetId.java
@@ -1,8 +1,12 @@
-package edu.uci.ics.asterix.transaction.management.service.transaction;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.io.Serializable;
 
 public class DatasetId implements Serializable {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
     int id;
 
     public DatasetId(int id) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/FileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
similarity index 98%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/FileBasedBuffer.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
index e88d74e..0e80817 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/FileBasedBuffer.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.logging;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.io.File;
 import java.io.IOException;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/FileUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
similarity index 95%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/FileUtil.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
index 46e03f1..2ba0a08 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/FileUtil.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.logging;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.io.File;
 import java.io.IOException;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IAsterixAppRuntimeContextProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
similarity index 76%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IAsterixAppRuntimeContextProvider.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
index d96d5c2..b53698b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IAsterixAppRuntimeContextProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
@@ -1,12 +1,12 @@
-package edu.uci.ics.asterix.transaction.management.service.recovery;
+package edu.uci.ics.asterix.common.transactions;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
@@ -18,7 +18,7 @@
 
     public IFileMapProvider getFileMapManager();
 
-    public TransactionSubsystem getTransactionSubsystem();
+    public ITransactionSubsystem getTransactionSubsystem();
 
     public IIndexLifecycleManager getIndexLifecycleManager();
 
@@ -26,11 +26,7 @@
 
     public ILSMMergePolicy getLSMMergePolicy();
 
-    public ILSMOperationTrackerFactory getLSMBTreeOperationTrackerFactory();
-
-    public ILSMOperationTrackerFactory getLSMRTreeOperationTrackerFactory();
-
-    public ILSMOperationTrackerFactory getLSMInvertedIndexOperationTrackerFactory();
+    public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID);
 
     public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider();
 
@@ -47,4 +43,6 @@
     public ResourceIdFactory getResourceIdFactory();
 
     public IIOManager getIOManager();
+
+    public IVirtualBufferCache getVirtualBufferCache(int datasetID);
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
similarity index 95%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IBuffer.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
index 8f39cbf..abb8ab8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IBuffer.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.logging;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.nio.ByteBuffer;
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ICloseable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
similarity index 81%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ICloseable.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
index f62fd66..4315a03 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ICloseable.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
@@ -13,10 +13,9 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.resource;
+package edu.uci.ics.asterix.common.transactions;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 
 /**
  * Represents a closeable resource that implements a close(@see
@@ -35,6 +34,6 @@
      * @param context
      * @throws ACIDException
      */
-    public void close(TransactionContext context) throws ACIDException;
+    public void close(ITransactionContext context) throws ACIDException;
 
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IFileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
similarity index 95%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IFileBasedBuffer.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
index a4ea3cb..5093ca0 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IFileBasedBuffer.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.logging;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.io.IOException;
 
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockHashTable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockHashTable.java
new file mode 100644
index 0000000..8c0e373
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockHashTable.java
@@ -0,0 +1,21 @@
+package edu.uci.ics.asterix.common.transactions;
+
+/**
+ * @author pouria Interface for a hashTable, used in the internal data
+ *         structures of lockManager
+ * @param <K>
+ *            Type of the objects, used as keys
+ * @param <V>
+ *            Type of the objects, used as values
+ */
+public interface ILockHashTable<K, V> {
+
+    public void put(K key, V value);
+
+    public V get(K key);
+
+    public V remove(K key);
+
+    public int getKeysetSize();
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
similarity index 75%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
index 1341cc1..79fbf3b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
@@ -12,18 +12,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.locking;
+package edu.uci.ics.asterix.common.transactions;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 
 /**
  * Interface for the lockManager
  * 
- * @author pouria 
+ * @author pouria
  * @author kisskys
- * 
  */
 public interface ILockManager {
 
@@ -40,13 +37,15 @@
      * has a "weaker" lock, then the request would be interpreted as a convert
      * request
      * Waiting transaction would eventually garb the lock, or get timed-out
+     * 
      * @param datasetId
      * @param entityHashValue
      * @param lockMode
      * @param txnContext
      * @throws ACIDException
      */
-    public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext) throws ACIDException;
+    public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
+            throws ACIDException;
 
     /**
      * The method releases "All" the locks taken/waiting-on by a specific
@@ -57,41 +56,45 @@
      * @param txnContext
      * @throws ACIDException
      */
-    public void releaseLocks(TransactionContext txnContext) throws ACIDException;
+    public void releaseLocks(ITransactionContext txnContext) throws ACIDException;
 
     /**
-     * 
      * @param datasetId
      * @param entityHashValue
      * @param txnContext
-     * @throws ACIDException TODO
+     * @throws ACIDException
+     *             TODO
+     * @return true if the lock count is 0, false otherwise.
      */
-    public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext) throws ACIDException;
+    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
+            throws ACIDException;
 
     /**
-     * 
      * @param datasetId
      * @param entityHashValue
      * @param txnContext
-     * @throws ACIDException TODO
+     * @throws ACIDException
+     *             TODO
+     * @return true if the lock count is 0, false otherwise.
      */
-    public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext, boolean commitFlag) throws ACIDException;
-    
+    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
+            throws ACIDException;
+
     /**
      * Call to lock and unlock a specific resource in a specific lock mode
+     * 
      * @param datasetId
      * @param entityHashValue
-     * @param lockMode TODO
+     * @param lockMode
+     *            TODO
      * @param context
-     * 
      * @return
      * @throws ACIDException
      */
-    public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext context) throws ACIDException;
-
+    public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext context)
+            throws ACIDException;
 
     /**
-     * 
      * @param datasetId
      * @param entityHashValue
      * @param lockMode
@@ -99,10 +102,10 @@
      * @return
      * @throws ACIDException
      */
-    public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext context) throws ACIDException;
-    
+    public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext context)
+            throws ACIDException;
+
     /**
-     * 
      * @param datasetId
      * @param entityHashValue
      * @param lockMode
@@ -110,8 +113,9 @@
      * @return
      * @throws ACIDException
      */
-    boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+    boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
             throws ACIDException;
+
     /**
      * Prints out the contents of the transactions' table in a readable fashion
      * 
@@ -120,6 +124,4 @@
      */
     public String prettyPrint() throws ACIDException;
 
-
-
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockMatrix.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockMatrix.java
new file mode 100644
index 0000000..e819b9a
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockMatrix.java
@@ -0,0 +1,30 @@
+package edu.uci.ics.asterix.common.transactions;
+
+/**
+ * @author pouria
+ *         Shows: - The conflict matrix for the locking protocol (whether two
+ *         lock modes conflict with each other or not on a single resource) -
+ *         Whether request to convert a lock mode to a new one is a conversion
+ *         (i.e. the new lock mode is stringer than the current one) or not
+ *         Each lock mode is shown/interpreted as an integer
+ */
+
+public interface ILockMatrix {
+
+    /**
+     * @param mask
+     *            (current/expected) lock mask on the resource
+     * @param reqLockMode
+     *            index of the requested lockMode
+     * @return true if the lock request conflicts with the mask
+     */
+    public boolean conflicts(int mask, int reqLockMode);
+
+    /**
+     * @param currentLockMode
+     * @param reqLockMode
+     * @return true if the request is a conversion
+     */
+    public boolean isConversion(int currentLockMode, int reqLockMode);
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogCursor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
similarity index 83%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogCursor.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
index 991de1b..ab2dcc2 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogCursor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
@@ -12,11 +12,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.logging;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.io.IOException;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 
 /**
  * Provides a cursor over the logs created to date.
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogFilter.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
similarity index 88%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogFilter.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
index 4c7eda2..9986add 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogFilter.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
@@ -12,7 +12,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.logging;
+package edu.uci.ics.asterix.common.transactions;
+
+import edu.uci.ics.asterix.common.transactions.IBuffer;
 
 /**
  * Represents a predicate or a filter that needs to be applied when selectively
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
similarity index 77%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
index 26229a7..c90422c 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
@@ -12,14 +12,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.logging;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.io.IOException;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 
 public interface ILogManager {
 
@@ -36,7 +33,7 @@
      * @param logicalLogLocator
      * @throws ACIDException
      */
-    void log(byte logType, TransactionContext context, int datasetId, int PKHashValue, long resourceId,
+    void log(byte logType, ITransactionContext context, int datasetId, int PKHashValue, long resourceId,
             byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
             LogicalLogLocator logicalLogLocator) throws ACIDException;
 
@@ -53,9 +50,10 @@
             ACIDException;
 
     /**
-     * @param logicalLogLocator TODO
-     * @param PhysicalLogLocator
-     *            specifies the location of the log record to be read
+     * @param lsnValue
+     *            TODO
+     * @param logicalLogLocator
+     *            TODO
      * @throws ACIDException
      */
     public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException;
@@ -83,6 +81,6 @@
      * 
      * @return TransactionSubsystem
      */
-    public TransactionSubsystem getTransactionSubsystem();
+    public ITransactionSubsystem getTransactionSubsystem();
 
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
similarity index 85%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
index 0e24f9d..13fc722 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
@@ -13,9 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+package edu.uci.ics.asterix.common.transactions;
 
 /**
  * Helper class for writing/reading of log header and checksum as well as
@@ -36,9 +34,9 @@
     public PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator);
 
     public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator);
-    
+
     public long getResourceId(LogicalLogLocator logicalLogLocator);
-    
+
     public byte getResourceMgrId(LogicalLogLocator logicalLogLocater);
 
     public int getLogContentSize(LogicalLogLocator logicalLogLocater);
@@ -51,8 +49,9 @@
 
     public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
 
-    public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, TransactionContext context, int datasetId,
-            int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId, int logRecordSize);
+    public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
+            int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
+            int logRecordSize);
 
     public boolean validateLogRecord(LogicalLogLocator logicalLogLocator);
 
@@ -61,7 +60,7 @@
     public int getLogHeaderSize(byte logType);
 
     public int getLogChecksumSize();
-    
+
     public int getCommitLogSize();
 
-}
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java
new file mode 100644
index 0000000..78dd50b
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.common.transactions;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+
+/**
+ * An interface providing call back APIs that are invoked {@link ILogManager} for providing the content for the log record and doing any pre/post
+ * processing.
+ */
+public interface ILogger {
+
+    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
+            throws ACIDException;
+
+    public void log(ITransactionContext context, final LogicalLogLocator logicalLogLocator, int logContentSize,
+            ReusableLogContentObject reusableLogContentObject) throws ACIDException;
+
+    public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
+            throws ACIDException;
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
new file mode 100644
index 0000000..323b9ce
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
@@ -0,0 +1,8 @@
+package edu.uci.ics.asterix.common.transactions;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+
+public interface ILoggerRepository {
+
+    public ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException;
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IRecoveryManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
similarity index 87%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IRecoveryManager.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
index 38802a2..dd9e734 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IRecoveryManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
@@ -12,12 +12,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.recovery;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.io.IOException;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 
 /**
  * Provides API for failure recovery. Failure could be at application level and
@@ -67,7 +67,7 @@
      *            the transaction context associated with the transaction
      * @throws ACIDException
      */
-    public void rollbackTransaction(TransactionContext txnContext) throws ACIDException;
+    public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException;
 
     public void checkpoint(boolean isSharpCheckpoint) throws ACIDException;
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
similarity index 88%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
index f7715e8..6874c7e 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
@@ -12,11 +12,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.transaction;
+package edu.uci.ics.asterix.common.transactions;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogRecordHelper;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 
 /**
  * Provides APIs for undo or redo of an operation on a resource.
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
new file mode 100644
index 0000000..80917e0
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
@@ -0,0 +1,60 @@
+package edu.uci.ics.asterix.common.transactions;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+
+public interface ITransactionContext {
+
+    public void registerIndexAndCallback(ILSMIndex index, AbstractOperationCallback callback);
+
+    public void updateLastLSNForIndexes(long lastLSN);
+
+    public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException;
+
+    public int getActiveOperationCountOnIndexes() throws HyracksDataException;
+
+    public LogicalLogLocator getFirstLogLocator();
+
+    public LogicalLogLocator getLastLogLocator();
+
+    public void addCloseableResource(ICloseable resource);
+
+    public JobId getJobId();
+
+    public void setStartWaitTime(long time);
+
+    public long getStartWaitTime();
+
+    public void setStatus(int status);
+
+    public int getStatus();
+
+    public void setTxnState(TransactionState txnState);
+
+    public TransactionState getTxnState();
+
+    public void releaseResources() throws ACIDException;
+
+    public void setLastLSN(long lsn);
+
+    public TransactionType getTransactionType();
+
+    public void setTransactionType(TransactionType transactionType);
+
+    public String prettyPrint();
+
+    public static final long INVALID_TIME = -1l; // used for showing a
+    // transaction is not waiting.
+    public static final int ACTIVE_STATUS = 0;
+    public static final int TIMED_OUT_STATUS = 1;
+
+    public enum TransactionType {
+        READ,
+        READ_WRITE
+    }
+
+    public void setExclusiveJobLevelCommit();
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
similarity index 70%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
index 3f55ac9..e57cc64 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
@@ -12,9 +12,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.transaction;
+package edu.uci.ics.asterix.common.transactions;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 
 /**
  * Provides APIs for managing life cycle of a transaction, that is beginning a
@@ -43,10 +43,10 @@
      * @param jobId
      *            a unique value for the transaction id.
      * @return the transaction context associated with the initiated transaction
-     * @see TransactionContext
+     * @see ITransactionContext
      * @throws ACIDException
      */
-    public TransactionContext beginTransaction(JobId jobId) throws ACIDException;
+    public ITransactionContext beginTransaction(JobId jobId) throws ACIDException;
 
     /**
      * Returns the transaction context of an active transaction given the
@@ -57,33 +57,39 @@
      * @return
      * @throws ACIDException
      */
-    public TransactionContext getTransactionContext(JobId jobId) throws ACIDException;
+    public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException;
 
     /**
      * Commits a transaction.
      * 
      * @param txnContext
      *            the transaction context associated with the transaction
-     * @param datasetId TODO
-     * @param PKHashVal TODO
+     * @param datasetId
+     *            TODO
+     * @param PKHashVal
+     *            TODO
      * @throws ACIDException
-     * @see transactionContextimport edu.uci.ics.hyracks.api.job.JobId;
+     * @see ITransactionContextimport edu.uci.ics.hyracks.api.job.JobId;
      * @see ACIDException
      */
-    public void commitTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal) throws ACIDException;
+    public void commitTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
+            throws ACIDException;
 
     /**
      * Aborts a transaction.
      * 
      * @param txnContext
      *            the transaction context associated with the transaction
-     * @param datasetId TODO
-     * @param PKHashVal TODO
+     * @param datasetId
+     *            TODO
+     * @param PKHashVal
+     *            TODO
      * @throws ACIDException
-     * @see transactionContext
+     * @see ITransactionContext
      * @see ACIDException
      */
-    public void abortTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal) throws ACIDException;
+    public void abortTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
+            throws ACIDException;
 
     /**
      * Indicates end of all activity for a transaction. In other words, all
@@ -92,23 +98,26 @@
      * 
      * @param txnContext
      *            the transaction context associated with the transaction
-     * @param datasetId TODO
-     * @param PKHashVal TODO
+     * @param datasetId
+     *            TODO
+     * @param PKHashVal
+     *            TODO
      * @param success
      *            indicates the success or failure. The transaction is committed
      *            or aborted accordingly.
      * @throws ACIDException
      */
-    public void completedTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal, boolean success) throws ACIDException;
+    public void completedTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal, boolean success)
+            throws ACIDException;
 
     /**
      * Returns the Transaction Provider for the transaction eco-system. A
      * transaction eco-system consists of a Log Manager, a Recovery Manager, a
      * Transaction Manager and a Lock Manager.
      * 
-     * @see TransactionSubsystem
+     * @see ITransactionSubsystem
      * @return TransactionProvider
      */
-    public TransactionSubsystem getTransactionProvider();
+    public ITransactionSubsystem getTransactionProvider();
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
new file mode 100644
index 0000000..e5462da
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
@@ -0,0 +1,21 @@
+package edu.uci.ics.asterix.common.transactions;
+
+
+public interface ITransactionSubsystem {
+
+    public ILogManager getLogManager();
+
+    public ILockManager getLockManager();
+
+    public ITransactionManager getTransactionManager();
+
+    public IRecoveryManager getRecoveryManager();
+
+    public TransactionalResourceManagerRepository getTransactionalResourceRepository();
+
+    public ILoggerRepository getTreeLoggerRepository();
+
+    public IAsterixAppRuntimeContextProvider getAsterixAppRuntimeContextProvider();
+
+    public String getId();
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/JobId.java
similarity index 90%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobId.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/JobId.java
index d306670..1246f22 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobId.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/JobId.java
@@ -1,4 +1,4 @@
-package edu.uci.ics.asterix.transaction.management.service.transaction;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.io.Serializable;
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
similarity index 60%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
index 581ce4c..9387687 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
@@ -12,31 +12,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.logging;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.io.Serializable;
-import java.util.Properties;
+
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
 
 public class LogManagerProperties implements Serializable {
 
     private static final long serialVersionUID = 2084227360840799662L;
 
     public static final int LOG_MAGIC_NUMBER = 123456789;
-    public static final String LOG_DIR_SUFFIX_KEY = ".txnLogDir";
-    public static final String LOG_PAGE_SIZE_KEY = "log_page_size";
-    public static final String LOG_PARTITION_SIZE_KEY = "log_partition_size";
-    public static final String NUM_LOG_PAGES_KEY = "num_log_pages";
-    public static final String LOG_FILE_PREFIX_KEY = "log_file_prefix";
-    public static final String GROUP_COMMIT_WAIT_PERIOD_KEY = "group_commit_wait_period";
-    public static final String DISK_SECTOR_SIZE_KEY = "disk_sector_size";
-
-    private static final int DEFAULT_LOG_PAGE_SIZE = 128 * 1024; //128KB
-    private static final int DEFAULT_NUM_LOG_PAGES = 8;
-    private static final long DEFAULT_LOG_PARTITION_SIZE = (long) 1024 * 1024 * 1024 * 2; //2GB 
-    private static final long DEFAULT_GROUP_COMMIT_WAIT_PERIOD = 200; // time in millisec.
+    public static final String LOG_DIR_SUFFIX = ".txnLogDir";
     private static final String DEFAULT_LOG_FILE_PREFIX = "asterix_transaction_log";
-    private static final String DEFAULT_LOG_DIRECTORY = "asterix_logs/";
-    private static final int DEFAULT_DISK_SECTOR_SIZE = 4096;
 
     // follow the naming convention <logFilePrefix>_<number> where number starts from 0
     private final String logFilePrefix;
@@ -56,22 +44,19 @@
     // default disk sector size
     private final int diskSectorSize;
 
-    public LogManagerProperties(Properties properties, String nodeId) {
-        this.logDirKey = new String(nodeId + LOG_DIR_SUFFIX_KEY);
-        this.logPageSize = Integer.parseInt(properties.getProperty(LOG_PAGE_SIZE_KEY, "" + DEFAULT_LOG_PAGE_SIZE));
-        this.numLogPages = Integer.parseInt(properties.getProperty(NUM_LOG_PAGES_KEY, "" + DEFAULT_NUM_LOG_PAGES));
-        long logPartitionSize = Long.parseLong(properties.getProperty(LOG_PARTITION_SIZE_KEY, ""
-                + DEFAULT_LOG_PARTITION_SIZE));
-        this.logDir = properties.getProperty(logDirKey, DEFAULT_LOG_DIRECTORY + nodeId);
-        this.logFilePrefix = properties.getProperty(LOG_FILE_PREFIX_KEY, DEFAULT_LOG_FILE_PREFIX);
-        this.groupCommitWaitPeriod = Long.parseLong(properties.getProperty(GROUP_COMMIT_WAIT_PERIOD_KEY, ""
-                + DEFAULT_GROUP_COMMIT_WAIT_PERIOD));
+    public LogManagerProperties(AsterixTransactionProperties txnProperties, String nodeId) {
+        this.logDirKey = new String(nodeId + LOG_DIR_SUFFIX);
+        this.logPageSize = txnProperties.getLogBufferPageSize();
+        this.numLogPages = txnProperties.getLogBufferNumPages();
+        long logPartitionSize = txnProperties.getLogPartitionSize();
+        this.logDir = txnProperties.getLogDirectory() + nodeId;
+        this.logFilePrefix = DEFAULT_LOG_FILE_PREFIX;
+        this.groupCommitWaitPeriod = txnProperties.getGroupCommitInterval();
 
         this.logBufferSize = logPageSize * numLogPages;
         //make sure that the log partition size is the multiple of log buffer size.
         this.logPartitionSize = (logPartitionSize / logBufferSize) * logBufferSize;
-        this.diskSectorSize = Integer.parseInt(properties.getProperty(DISK_SECTOR_SIZE_KEY, ""
-                + DEFAULT_DISK_SECTOR_SIZE));
+        this.diskSectorSize = txnProperties.getLogDiskSectorSize();
     }
 
     public long getLogPartitionSize() {
@@ -105,7 +90,7 @@
     public String getLogDirKey() {
         return logDirKey;
     }
-    
+
     public int getDiskSectorSize() {
         return diskSectorSize;
     }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java
new file mode 100644
index 0000000..acf3346
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java
@@ -0,0 +1,136 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.common.transactions;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+
+/**
+ * A utility class providing helper methods for the {@link ILogManager}
+ */
+public class LogUtil {
+
+    private static final Logger LOGGER = Logger.getLogger(LogUtil.class.getName());
+
+    // read the log directory and initialize log anchor to point to the
+    // current log partition file and the offset where the log manager shall
+    // continue to insert log records.
+
+    public static PhysicalLogLocator initializeLogAnchor(ILogManager logManager) throws ACIDException {
+        int fileId = 0;
+        long offset = 0;
+        LogManagerProperties logManagerProperties = logManager.getLogManagerProperties();
+        File logDir = new File(logManagerProperties.getLogDir());
+        try {
+            if (logDir.exists()) {
+                List<String> logFiles = getLogFiles(logManagerProperties);
+                if (logFiles == null || logFiles.size() == 0) {
+                    FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
+                    }
+                } else {
+                    File logFile = new File(LogUtil.getLogFilePath(logManagerProperties,
+                            Long.parseLong(logFiles.get(logFiles.size() - 1))));
+                    fileId = logFiles.size() - 1;
+                    offset = logFile.length();
+                }
+            } else {
+                FileUtil.createNewDirectory(logManagerProperties.getLogDir());
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
+                }
+                FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
+                }
+            }
+        } catch (IOException ioe) {
+            throw new ACIDException("Unable to initialize log anchor", ioe);
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(" file id :" + fileId + " offset " + offset);
+        }
+        return new PhysicalLogLocator(fileId, offset, logManager);
+    }
+
+    public static List<String> getLogFiles(final LogManagerProperties logManagerProperties) {
+        File logDir = new File(logManagerProperties.getLogDir());
+        String[] logFiles = new String[0];
+        List<String> logFileCollection = new ArrayList<String>();
+        if (logDir.exists()) {
+            logFiles = logDir.list(new FilenameFilter() {
+                public boolean accept(File dir, String name) {
+                    if (name.startsWith(logManagerProperties.getLogFilePrefix())) {
+                        return true;
+                    }
+                    return false;
+                }
+            });
+        }
+        for (String logFile : logFiles) {
+            logFileCollection.add(logFile.substring(logManagerProperties.getLogFilePrefix().length() + 1));
+        }
+        Collections.sort(logFileCollection, new Comparator<String>() {
+            @Override
+            public int compare(String arg0, String arg1) {
+                return Integer.parseInt(arg0) - Integer.parseInt(arg1);
+            }
+        });
+        return logFileCollection;
+    }
+
+    public static long getFileId(String logFilePath, LogManagerProperties logManagerProperties) {
+        String logFileName = logFilePath;
+        if (logFilePath.contains(File.separator)) {
+            logFileName = logFilePath.substring(logFilePath.lastIndexOf(File.separator));
+        }
+        return Long.parseLong(logFileName.substring(logFileName.indexOf(logManagerProperties.getLogFilePrefix())));
+    }
+
+    public static String getLogFilePath(LogManagerProperties logManagerProperties, long fileId) {
+        return logManagerProperties.getLogDir() + File.separator + logManagerProperties.getLogFilePrefix() + "_"
+                + fileId;
+    }
+
+    public static LogicalLogLocator getDummyLogicalLogLocator(ILogManager logManager) {
+        LogicalLogLocator logicalLogLocator = new LogicalLogLocator(-1, null, -1, logManager);
+        return logicalLogLocator;
+    }
+
+    /*
+     * given a lsn, get the offset within the log file where the corresponding
+     * log record is (to be) placed.
+     */
+    public static long getFileOffset(ILogManager logManager, long lsn) {
+        return lsn % logManager.getLogManagerProperties().getLogPartitionSize();
+    }
+
+    /*
+     * given a lsn, get the file id that contains the log record.
+     */
+    public static long getFileId(ILogManager logManager, long lsn) {
+        return lsn / logManager.getLogManagerProperties().getLogPartitionSize();
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogicalLogLocator.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
similarity index 96%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogicalLogLocator.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
index f9eec7d..81dc6e2 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogicalLogLocator.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.logging;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.util.concurrent.atomic.AtomicInteger;
 
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java
new file mode 100644
index 0000000..a6cebd6
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java
@@ -0,0 +1,30 @@
+package edu.uci.ics.asterix.common.transactions;
+
+public class MutableResourceId{
+    long id;
+
+    public MutableResourceId(long id) {
+        this.id = id;
+    }
+
+    public void setId(long id) {
+        this.id = id;
+    }
+
+    public long getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int)id;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if ((o == null) || !(o instanceof MutableResourceId)) {
+            return false;
+        }
+        return ((MutableResourceId) o).id == this.id;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PhysicalLogLocator.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
similarity index 96%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PhysicalLogLocator.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
index d8de007..7c0cff7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PhysicalLogLocator.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.transaction.management.service.logging;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.util.concurrent.atomic.AtomicLong;
 
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java
new file mode 100644
index 0000000..a327c10
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java
@@ -0,0 +1,67 @@
+package edu.uci.ics.asterix.common.transactions;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+
+/**
+ * Represents the state of a transaction thread. The state contains information
+ * that includes the tuple being operated, the operation and the location of the
+ * log record corresponding to the operation.
+ */
+public class ReusableLogContentObject {
+
+    private LogicalLogLocator logicalLogLocator;
+    private IndexOperation newOperation;
+    private ITupleReference newValue;
+    private IndexOperation oldOperation;
+    private ITupleReference oldValue;
+
+    public ReusableLogContentObject(LogicalLogLocator logicalLogLocator, IndexOperation newOperation,
+            ITupleReference newValue, IndexOperation oldOperation, ITupleReference oldValue) {
+        this.logicalLogLocator = logicalLogLocator;
+        this.newOperation = newOperation;
+        this.newValue = newValue;
+        this.oldOperation = oldOperation;
+        this.oldValue = oldValue;
+    }
+
+    public synchronized LogicalLogLocator getLogicalLogLocator() {
+        return logicalLogLocator;
+    }
+
+    public synchronized void setLogicalLogLocator(LogicalLogLocator logicalLogLocator) {
+        this.logicalLogLocator = logicalLogLocator;
+    }
+
+    public synchronized void setNewOperation(IndexOperation newOperation) {
+        this.newOperation = newOperation;
+    }
+
+    public synchronized IndexOperation getNewOperation() {
+        return newOperation;
+    }
+
+    public synchronized void setNewValue(ITupleReference newValue) {
+        this.newValue = newValue;
+    }
+
+    public synchronized ITupleReference getNewValue() {
+        return newValue;
+    }
+
+    public synchronized void setOldOperation(IndexOperation oldOperation) {
+        this.oldOperation = oldOperation;
+    }
+
+    public synchronized IndexOperation getOldOperation() {
+        return oldOperation;
+    }
+
+    public synchronized void setOldValue(ITupleReference oldValue) {
+        this.oldValue = oldValue;
+    }
+
+    public synchronized ITupleReference getOldValue() {
+        return oldValue;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceManagerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
similarity index 92%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceManagerRepository.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
index d38226b..35f69d4 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceManagerRepository.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
@@ -13,13 +13,11 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.asterix.transaction.management.resource;
+package edu.uci.ics.asterix.common.transactions;
 
 import java.util.HashMap;
 import java.util.Map;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
-
 /**
  * Represents a repository containing Resource Managers and Resources in the
  * transaction eco-system. Operations on a resource require acquiring
diff --git a/asterix-common/src/main/resources/schema/asterix-conf.xsd b/asterix-common/src/main/resources/schema/asterix-conf.xsd
index f53fb4b..5aefdbd 100644
--- a/asterix-common/src/main/resources/schema/asterix-conf.xsd
+++ b/asterix-common/src/main/resources/schema/asterix-conf.xsd
@@ -7,6 +7,7 @@
 
         
 	<xs:element name="metadataNode" type="xs:string" />
+	<xs:element name="coredumpPath" type="xs:string" />
 	<xs:element name="storeDirs" type="xs:string" />
 	<xs:element name="ncId" type="xs:string" />
 	<xs:element name="name" type="xs:string" />
@@ -23,6 +24,15 @@
 		</xs:complexType>
 	</xs:element>
 
+	<xs:element name="coredump">
+		<xs:complexType>
+			<xs:sequence>
+				<xs:element ref="mg:ncId" />
+				<xs:element ref="mg:coredumpPath" />
+			</xs:sequence>
+		</xs:complexType>
+	</xs:element>
+
 	<xs:element name="property">
 		<xs:complexType>
 			<xs:sequence>
@@ -39,6 +49,7 @@
 			<xs:sequence>
 				<xs:element ref="mg:metadataNode" minOccurs="0"/>
 				<xs:element ref="mg:store" maxOccurs="unbounded" />
+				<xs:element ref="mg:coredump" maxOccurs="unbounded" />
 				<xs:element ref="mg:property" minOccurs="0" maxOccurs="unbounded" />
 			</xs:sequence>
 		</xs:complexType>
diff --git a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
index 8449971..d13dd2e 100644
--- a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
+++ b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
@@ -9,13 +9,13 @@
 import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.util.Iterator;
 import java.util.List;
-import java.util.NoSuchElementException;
+import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.commons.httpclient.DefaultHttpMethodRetryHandler;
@@ -24,10 +24,15 @@
 import org.apache.commons.httpclient.NameValuePair;
 import org.apache.commons.httpclient.methods.GetMethod;
 import org.apache.commons.httpclient.params.HttpMethodParams;
-import org.json.JSONArray;
+import org.codehaus.jackson.map.JsonMappingException;
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+
+import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.testframework.context.TestCaseContext;
 import edu.uci.ics.asterix.testframework.context.TestFileContext;
 import edu.uci.ics.asterix.testframework.xml.TestCase.CompilationUnit;
@@ -135,84 +140,46 @@
         return fname.substring(0, dot + 1) + EXTENSION_AQL_RESULT;
     }
 
-    public static void writeResultsToFile(File actualFile, JSONObject result) throws IOException, JSONException {
+    public static void writeResultsToFile(File actualFile, InputStream resultStream) throws IOException, JSONException {
         BufferedWriter writer = new BufferedWriter(new FileWriter(actualFile));
-        Results res = new Results(result);
-        for (String line : res) {
-            writer.write(line);
-            writer.newLine();
-        }
-        writer.close();
-    }
-
-    public static class Results implements Iterable<String> {
-        private final JSONArray chunks;
-
-        public Results(JSONObject result) throws JSONException {
-            chunks = result.getJSONArray("results");
-        }
-
-        public Iterator<String> iterator() {
-            return new ResultIterator(chunks);
+        try {
+            JsonFactory jsonFactory = new JsonFactory();
+            JsonParser resultParser = jsonFactory.createParser(resultStream);
+            while (resultParser.nextToken() == JsonToken.START_OBJECT) {
+                while (resultParser.nextToken() != JsonToken.END_OBJECT) {
+                    String key = resultParser.getCurrentName();
+                    if (key.equals("results")) {
+                        // Start of array.
+                        resultParser.nextToken();
+                        while (resultParser.nextToken() != JsonToken.END_ARRAY) {
+                            String record = resultParser.getValueAsString();
+                            writer.write(record);
+                        }
+                    } else {
+                        String summary = resultParser.getValueAsString();
+                        if (key.equals("summary")) {
+                            writer.write(summary);
+                            throw new JsonMappingException("Could not find results key in the JSON Object");
+                        }
+                    }
+                }
+            }
+        } finally {
+            writer.close();
         }
     }
 
-    public static class ResultIterator implements Iterator<String> {
-        private final JSONArray chunks;
-
-        private int chunkCounter = 0;
-        private int recordCounter = 0;
-
-        public ResultIterator(JSONArray chunks) {
-            this.chunks = chunks;
-        }
-
-        @Override
-        public boolean hasNext() {
-            JSONArray resultArray;
-            try {
-                resultArray = chunks.getJSONArray(chunkCounter);
-                if (resultArray.getString(recordCounter) != null) {
-                    return true;
-                }
-            } catch (JSONException e) {
-                return false;
-            }
-            return false;
-        }
-
-        @Override
-        public String next() throws NoSuchElementException {
-            JSONArray resultArray;
-            String item = "";
-
-            try {
-                resultArray = chunks.getJSONArray(chunkCounter);
-                item = resultArray.getString(recordCounter);
-                if (item == null) {
-                    throw new NoSuchElementException();
-                }
-                item = item.trim();
-
-                recordCounter++;
-                if (recordCounter >= resultArray.length()) {
-                    chunkCounter++;
-                    recordCounter = 0;
-                }
-            } catch (JSONException e) {
-                throw new NoSuchElementException(e.getMessage());
-            }
-            return item;
-        }
-
-        @Override
-        public void remove() {
-            throw new UnsupportedOperationException();
-        }
+    private static String[] handleError(GetMethod method) throws Exception {
+        String errorBody = method.getResponseBodyAsString();
+        JSONObject result = new JSONObject(errorBody);
+        String[] errors = { result.getJSONArray("error-code").getString(0), result.getString("summary"),
+                result.getString("stacktrace") };
+        return errors;
     }
 
     // Executes Query and returns results as JSONArray
-    public static JSONObject executeQuery(String str) throws Exception {
+    public static InputStream executeQuery(String str) throws Exception {
+        InputStream resultStream = null;
 
         final String url = "http://localhost:19101/query";
 
@@ -227,26 +194,22 @@
         // Provide custom retry handler is necessary
         method.getParams().setParameter(HttpMethodParams.RETRY_HANDLER, new DefaultHttpMethodRetryHandler(3, false));
 
-        JSONObject result = null;
-
         try {
             // Execute the method.
             int statusCode = client.executeMethod(method);
 
             // Check if the method was executed successfully.
             if (statusCode != HttpStatus.SC_OK) {
-                System.err.println("Method failed: " + method.getStatusLine());
+                GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, "Method failed: " + method.getStatusLine());
             }
 
-            // Read the response body as String.
-            String responseBody = method.getResponseBodyAsString();
-
-            result = new JSONObject(responseBody);
+            // Read the response body as stream
+            resultStream = method.getResponseBodyAsStream();
         } catch (Exception e) {
-            System.out.println(e.getMessage());
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, e.getMessage(), e);
             e.printStackTrace();
         }
-        return result;
+        return resultStream;
     }
 
     // To execute Update statements
@@ -270,7 +233,10 @@
 
         // Check if the method was executed successfully.
         if (statusCode != HttpStatus.SC_OK) {
-            System.err.println("Method failed: " + method.getStatusLine());
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, "Method failed: " + method.getStatusLine());
+            String[] errors = handleError(method);
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, errors[2]);
+            throw new Exception("DML operation failed: " + errors[0]);
         }
     }
 
@@ -299,7 +265,10 @@
 
         // Check if the method was executed successfully.
         if (statusCode != HttpStatus.SC_OK) {
-            System.err.println("Method failed: " + method.getStatusLine());
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, "Method failed: " + method.getStatusLine());
+            String[] errors = handleError(method);
+            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, errors[2]);
+            throw new Exception("DDL operation failed: " + errors[0]);
         }
     }
 
@@ -342,7 +311,6 @@
 
         List<CompilationUnit> cUnits = testCaseCtx.getTestCase().getCompilationUnit();
         for (CompilationUnit cUnit : cUnits) {
-            LOGGER.info("[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName());
 
             testFileCtxs = testCaseCtx.getTestFiles(cUnit);
             expectedResultFileCtxs = testCaseCtx.getExpectedResultFiles(cUnit);
@@ -359,25 +327,24 @@
                             TestsUtils.executeUpdate(statement);
                             break;
                         case "query":
-                            result = TestsUtils.executeQuery(statement);
-                            if (!cUnit.getExpectedError().isEmpty()) {
-                                if (!result.has("error")) {
-                                    throw new Exception("Test \"" + testFile + "\" FAILED!");
-                                }
-                            } else {
+                            try {
+                                InputStream resultStream = executeQuery(statement);
                                 expectedResultFile = expectedResultFileCtxs.get(queryCount).getFile();
 
                                 File actualFile = new File(actualPath + File.separator
                                         + testCaseCtx.getTestCase().getFilePath().replace(File.separator, "_") + "_"
                                         + cUnit.getName() + ".adm");
+                                TestsUtils.writeResultsToFile(actualFile, resultStream);
 
                                 File actualResultFile = testCaseCtx.getActualResultFile(cUnit, new File(actualPath));
                                 actualResultFile.getParentFile().mkdirs();
 
-                                TestsUtils.writeResultsToFile(actualFile, result);
-
                                 TestsUtils.runScriptAndCompareWithResult(testFile, new PrintWriter(System.err),
                                         expectedResultFile, actualFile);
+                                LOGGER.info("[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/"
+                                        + cUnit.getName() + " PASSED ");
+                            } catch (JsonMappingException e) {
+                                throw new Exception("Test \"" + testFile + "\" FAILED!\n");
                             }
                             queryCount++;
                             break;
@@ -387,6 +354,7 @@
                         default:
                             throw new IllegalArgumentException("No statements of type " + ctx.getType());
                     }
+
                 } catch (Exception e) {
                     if (cUnit.getExpectedError().isEmpty()) {
                         throw new Exception("Test \"" + testFile + "\" FAILED!", e);
@@ -394,6 +362,6 @@
                 }
             }
         }
-
     }
 }
+
diff --git a/asterix-doc/pom.xml b/asterix-doc/pom.xml
index d987e5f..4b8cb2f 100644
--- a/asterix-doc/pom.xml
+++ b/asterix-doc/pom.xml
@@ -1,21 +1,21 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-	<parent>
-		<artifactId>asterix</artifactId>
-		<groupId>edu.uci.ics.asterix</groupId>
-		<version>0.0.6-SNAPSHOT</version>
-	</parent>
-	<artifactId>asterix-doc</artifactId>
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-site-plugin</artifactId>
-				<version>3.3</version>
-				<configuration>
-				  <generateReports>false</generateReports>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>asterix</artifactId>
+    <groupId>edu.uci.ics.asterix</groupId>
+    <version>0.0.6-SNAPSHOT</version>
+  </parent>
+  <artifactId>asterix-doc</artifactId>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <version>3.3</version>
+        <configuration>
+          <generateReports>false</generateReports>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
 </project>
diff --git a/asterix-doc/src/site/markdown/AsterixAlphaRelease.md b/asterix-doc/src/site/markdown/AsterixAlphaRelease.md
deleted file mode 100644
index ff9efb2..0000000
--- a/asterix-doc/src/site/markdown/AsterixAlphaRelease.md
+++ /dev/null
@@ -1,57 +0,0 @@
-# AsterixDB: A Big Data Management System _(Alpha Release)_ #
-
-## What Is AsterixDB? ##
-
-Welcome to the new home of the AsterixDB Big Data Management System (BDMS).
-The AsterixDB BDMS is the result of about 3.5 years of R&D involving researchers at UC Irvine, UC Riverside, and UC San Diego.
-The AsterixDB code base now consists of roughly 250K lines of Java code that has been co-developed at UC Irvine and UC Riverside.
-
-Initiated in 2009, the NSF-sponsored ASTERIX project has been developing new technologies for ingesting, storing, managing, indexing, querying, and analyzing vast quantities of semi-structured information.
-The project has been combining ideas from three distinct areas---semi-structured data, parallel databases, and data-intensive computing (a.k.a. today's Big Data platforms)---in order to create a next-generation, open-source software platform that scales by running on large, shared-nothing commodity computing clusters.
-The ASTERIX effort has been targeting a wide range of semi-structured information, ranging from "data" use cases---where information is well-typed and highly regular---to "content" use cases---where data tends to be irregular, much of each datum may be textual, and the ultimate schema for the various data types involved may be hard to anticipate up front.
-The ASTERIX project has been addressing technical issues including highly scalable data storage and indexing, semi-structured query processing on very large clusters, and merging time-tested parallel database techniques with modern data-intensive computing techniques to support performant yet declarative solutions to the problem of storing and analyzing semi-structured information effectively.
-The first fruits of this labor have been captured in the AsterixDB system that is now being released in preliminary or "Alpha" release form.
-We are hoping that the arrival of AsterixDB will mark the beginning of the "BDMS era", and we hope that both the Big Data community and the database community will find the AsterixDB system to be interesting and useful for a much broader class of problems than can be addressed with any one of today's current Big Data platforms and related technologies (e.g., Hadoop, Pig, Hive, HBase, MongoDB, and so on).  One of our project mottos has been "one size fits a bunch"---at least that has been our aim.  For more information about the research effort that led to the birth of AsterixDB, please refer to our NSF project web site: [http://asterix.ics.uci.edu/](http://asterix.ics.uci.edu/).
-
-In a nutshell, AsterixDB is a full-function BDMS with a rich feature set that distinguishes it from pretty much any other Big Data platform that's out and available today.  We believe that its feature set makes it well-suited to modern needs such as web data warehousing and social data storage and analysis.  AsterixDB has:
-
- * A semistructured NoSQL style data model (ADM) resulting from extending JSON with object database ideas
- * An expressive and declarative query language (AQL) that supports a broad range of queries and analysis over semistructured data
- * A parallel runtime query execution engine, Hyracks, that has been scale-tested on up to 1000+ cores and 500+ disks
- * Partitioned LSM-based data storage and indexing to support efficient ingestion and management of semistructured data
- * Support for query access to externally stored data (e.g., data in HDFS) as well as to data stored natively by AsterixDB
- * A rich set of primitive data types, including spatial and temporal data in addition to integer, floating point, and textual data
- * Secondary indexing options that include B+ trees, R trees, and inverted keyword (exact and fuzzy) index types
- * Support for fuzzy and spatial queries as well as for more traditional parametric queries
- * Basic transactional (concurrency and recovery) capabilities akin to those of a NoSQL store
-
-## Getting and Using AsterixDB ##
-
-You are most likely here because you are interested in getting your hands on AsterixDB---so you would like to know how to get it, how to set it up, and how to use it.
-Someday our plan is to have comprehensive documentation for AsterixDB and its data model (ADM) and query language (AQL) here on this wiki.
-For the Alpha release, we've got a start; for the Beta release a month or so from now, we will hopefully have much more.
-The following is a list of the wiki pages and supporting documents that we have available today:
-
-1. [InstallingAsterixUsingManagix](InstallingAsterixUsingManagix.html) :
-This is our installation guide, and it is where you should start.
-This document will tell you how to obtain, install, and manage instances of [AsterixDB](https://asterixdb.googlecode.com/files/asterix-installer-0.0.4-binary-assembly.zip), including both single-machine setup (for developers) as well as cluster installations (for deployment in its intended form).
-
-2. [AdmAql101](AdmAql101.html) :
-This is a first-timers introduction to the user model of the AsterixDB BDMS, by which we mean the view of AsterixDB as seen from the perspective of an "average user" or Big Data application developer.
-The AsterixDB user model consists of its data modeling features (ADM) and its query capabilities (AQL).
-This document presents a tiny "social data warehousing" example and uses it as a backdrop for describing, by example, the key features of AsterixDB.
-By working through this document, you will learn how to define the artifacts needed to manage data in AsterixDB, how to load data into the system, how to use most of the basic features of its query language, and how to insert and delete data dynamically.
-
-3. [AsterixDataTypesAndFunctions](AsterixDataTypesAndFunctions.html) :
-This is a reference document that catalogs the primitive data types and built-in functions available for use in AsterixDB schemas (in ADM) and queries (in AQL).
-
-4. [AQL Reference](AsterixQueryLanguageReference.html) :
-This is the AQL language reference manual.
-
-5. [AsterixDBRestAPI](AsterixDBRestAPI.html) :
-Access to data in an AsterixDB instance is provided via a REST-based API.
-This is a short document that describes the REST API entry points and their URL syntax.
-
-To all who have now come this far: Thanks for your interest in AsterixDB, and for kicking its tires in its Alpha form
-In addition to getting the system and trying it out, please sign up as a member of the AsterixDB user mailing list (asterixdb-users (at) googlegroups.com) so that you can contact us easily with your questions, issues, and other feedback.
-We want AsterixDB to be a "big hit" some day, and we are anxious to see what users do with it and to learn from that feedback what we should be working on most urgently in the next phase of the project.
diff --git a/asterix-doc/src/site/markdown/AsterixQueryLanguage.md b/asterix-doc/src/site/markdown/AsterixQueryLanguage.md
deleted file mode 100644
index af25cda..0000000
--- a/asterix-doc/src/site/markdown/AsterixQueryLanguage.md
+++ /dev/null
@@ -1,195 +0,0 @@
-`<wiki:toc max_depth="2" />`
-
-# The Asterix Query Language, Version 1.0 #
-
-# Introduction #
-
-This wiki page provides an overview of the Asterix Query language and the Asterix Data model.
-
-*WARNING:* _THIS IS AN INCOMPLETE SUSPENDED WORK IN PROGRESS...
-_  It will hopefully be resumed shortly in order to produce a legit AQL spec to go out with the Beta Release of AsterixDB.  What's here is very likely inconsistent with what's in the system as of today, as this was from an older snapshot of the world.
-
-# Asterix Data Model #
-
-Data in Asterix is represented using the Asterix Data Model (ADM). The ADM derives inspiration from prior standards such as JSON, XQuery, and the Object Data Model from ODMG.
-
-## Asterix Types ##
-
-### Primitive Types ##
-
-|| *Primitive Type* || *Description* ||
-|| int8   || Signed 8-bit integer. Valid range -128 thru 127 ||
-|| int16  || Signed 16-bit integer. Valid range -32768 thru 32767 ||
-|| int32  || Signed 32-bit integer. Valid range -2147483648 thru 2147483647 ||
-|| int64  || Signed 64-bit integer. Valid range -9223372036854775807 thru 9223372036854775808 ||
-|| uint8  || Unsigned 8-bit integer. Valid range 0 thru 255 ||
-|| uint16 || Unsigned 16-bit integer. Valid range 0 thru 65535 ||
-|| uint32 || Unsigned 32-bit integer. Valid range 0 thru 4294967295 ||
-|| uint64 || Unsigned 64-bit integer. Valid range 0 thru 18446744073709551615 ||
-|| string || String of characters ||
-|| null   || null type (Type of the null value) ||
-|| date   || Date ||
-|| time   || Time of day ||
-|| boolean || Boolean ||
-|| datetime || Date and time ||
-|| point2d || A point in 2-D space ||
-|| point3d || A point in 3-D space ||
-|| binary || Binary data ||
-|| yminterval || Year-Month interval ||
-|| dtinterval || Day-Time interval ||
-|| interval || Year-Month and Day-Time interval ||
-
-### Collection Types ###
-
-|| *Collection Type* || *Description* ||
-|| Record || A record type describes the record data item. A record contains a set of fields which can have values of any ADM type. Fields of a record must be unique. ||
-|| Union || A union type is an abstract type (A value never has a union type) that describes a set of type choices. ||
-|| Ordered List || An orderedlist instance represents a sequence of values where the order of the instances is determined by creation/insertion ||
-|| UnorderedList || An unorderedlist instance represents a collection of values where the order of the instances where the order is irrelevant ||
-|| Enumeration || An enumeration type represents a choice of string values ||
-
-# AQL Expressions #
-
-## Primary Expressions ##
-
-Primary expressions are the basic expressions that form the core of AQL.
-
-### Literals ###
-
-A Literal is a syntactic representation of a constant value. The various literals allowed in AQL are described in the table below.
-
-|| *Literal type* || *Syntax* ||
-|| StringLiteral || ` STRING_LITERAL : ("\"" ("\\\"" | ~["\""])* "\"") | ("\'"("\\\'" | ~["\'"])* "\'") ` ||
-|| IntegerLiteral || ` INTEGER_LITERAL : (["0" - "9"])+ ` ||
-|| FloatLiteral || ` FLOAT_LITERAL: ((["0" - "9"])* "." (["0" - "9"])+ ("f" | "F")) ` ||
-|| DoubleLiteral || ` DOUBLE_LITERAL: ((["0" - "9"])* "." (["0" - "9"])+) ` ||
-|| NullLiteral || ` NULL_LITERAL: "null" ` ||
-|| BooleanLiteral || ` BOOLEAN_LITERAL: "true" | "false" ` ||
-
-### Function Call ###
-
-Function Calls in AQL can be used to invoke builtin functions as well as user defined functions.
-Function Calls have the following syntax.
-
-
-            IDENTIFIER "(" ( Expression ( "," Expression )* )? ")"
-
-
-### Variable Reference ###
-
-Variables in AQL are used to bind to values. Variables can be bound to values by the For, Let, Group by clauses of the FLWOR expressions. Variables can also be bound by
-the Quantified Expressions.
-
-### Ordered List Constructor ###
-
-Constructs an ordered list. An ordered list represents a collection of values. The order of values is relevant. The collection may contain duplicate values.
-
-### Unordered List Constructor ###
-
-Constructs an unordered list. An unordered list represents a collection of values. The order of values is not relevant. The collection may contain duplicate values.
-
-### Record Constructor ###
-
-Constructs an AQL Record. A record contains fields. Each field has a name and a value. The name of the field is of type string. The value of a field may be any legal ADM data type. A record may not contain duplicate fields.
-
-## Arithmetic Expressions ##
-
-AQL allows all the standard arithmetic operators on numeric data types. The specific operators allowed are:
-
-|| *Operator* || *Description* ||
-|| + || Add ||
-|| - || Subtract ||
-|| * || Multiply ||
-|| / || Divide ||
-|| mod || Modulo ||
-
-## Comparison Expressions ##
-
-AQL provides the six standard comparison expressions listed below. In addition, AQL supports fuzzy comparisons.
-
-|| *Operator* || *Description* ||
-|| = || Equal ||
-|| = || Not Equal ||
-|| `< || Less Than ||
-|| `<= || Less Than or Equal ||
-|| >` || Greater Than ||
-|| >`= || Greater Than or Equal ||
-|| >`= || Greater Than or Equal ||
-|| ~= || Fuzzy Equals ||
-
-## Logical Expressions ##
-
-AQL provides two logical connectors:
-
-|| *Operator* || *Description* ||
-|| and || Logical AND ||
-|| or || Logical OR ||
-
-## Field Access Expressions ##
-
-The "." operator is used to access fields of a record. For example,
-
-
-            $x.name
-
-
-accesses the name field of the record bound to $x.
-
-## Indexed Expressions ##
-
-Indexed expressions are used to access values in an ordered list. For example,
-
-
-            $x[5]
-
-
-accesses the 6th item in the list bound to $x. Indexes start at 0.
-
-## FLWOR Expression ##
-
-The FLWOR expression is the most elaborate expression in AQL. It is made up of two parts -- Clauses and the Return Expression.
-
-The syntax of the FLWOR expression is:
-
-
-        
-        ( ForClause | LetClause )
-        ( ForClause | LetClause | WhereClause | OrderClause | GroupClause | LimitClause | DistinctClause )*
-        "return" ReturnExpression
-        
-
-
-* For Clause
-
-        "for" Variable "in" Expression
-
-* Let Clause
-
-        "let" Variable ":=" Expression
-
-* Where Clause
-
-        "where" Expression
-
-* Order Clause
-
-        "order" "by" Expression ("asc" | "desc") ("," Expression ("asc" | "desc"))*
-
-* Group Clause
-
-        "group" "by" ((Variable ":=")? Expression) ("," ((Variable ":=")? Expression))* "with" Variable
-
-* Limit Clause
-
-        "limit" Expression ("," Expression)?
-
-
-* If Then Else Expressions *
-
-        "if" "(" Expression ")" "then" Expression "else" Expression
-
-
-* Quantified Expressions *
-
-        ("some" | "every") Variable "in" Expression "satisfies" Expression
-
diff --git a/asterix-doc/src/site/markdown/AsterixQueryLanguageReference.md b/asterix-doc/src/site/markdown/AsterixQueryLanguageReference.md
deleted file mode 100644
index 1ed876e..0000000
--- a/asterix-doc/src/site/markdown/AsterixQueryLanguageReference.md
+++ /dev/null
@@ -1,462 +0,0 @@
-# The Asterix Query Language, Version 1.0
-## 1. Introduction
-
-This document provides an overview of the Asterix Query language.
-
-
-## 2. Expressions
-
-    Expression ::= ( OperatorExpr | IfThenElse | FLWOGR | QuantifiedExpression )
-
-
-### Primary Expressions
-
-    PrimaryExpr ::= Literal
-                  | VariableRef
-                  | ParenthesizedExpression
-                  | FunctionCallExpr
-                  | DatasetAccessExpression
-                  | ListConstructor
-                  | RecordConstructor
-                  
-
-#### Literals
-
-    Literal ::= StringLiteral
-              | <INTEGER_LITERAL>
-              | <FLOAT_LITERAL>
-              | <DOUBLE_LITERAL>
-              | "null"
-              | "true"
-              | "false"
-    StringLiteral ::= <STRING_LITERAL>
-
-##### Examples
-
-    "a string"
-    42
-
-
-#### Variable References
-
-    VariableRef ::= <VARIABLE>
-
-##### Example
-
-    $id  
-    
-
-#### Parenthesized Expressions
-    
-    ParenthesizedExpression ::= "(" Expression ")"
-
-##### Example
-
-    ( 1 + 1 )
-
-
-#### Function Calls
-
-    FunctionCallExpr ::= FunctionOrTypeName "(" ( Expression ( "," Expression )* )? ")"
-
-##### Example
-
-    string-length("a string")
-
-
-#### Dataset Access
-
-    DatasetAccessExpression ::= "dataset" ( ( Identifier ( "." Identifier )? )
-                              | ( "(" Expression ")" ) )
-    Identifier              ::= <IDENTIFIER> | StringLiteral
-
-##### Examples
-
-    dataset customers
-    dataset (string-join("customers", $country))
-    
-
-#### Constructors
-
-    ListConstructor          ::= ( OrderedListConstructor | UnorderedListConstructor )
-    OrderedListConstructor   ::= "[" ( Expression ( "," Expression )* )? "]"
-    UnorderedListConstructor ::= "{{" ( Expression ( "," Expression )* )? "}}"
-    RecordConstructor        ::= "{" ( FieldBinding ( "," FieldBinding )* )? "}"
-    FieldBinding             ::= Expression ":" Expression
-
-##### Examples
-
-    [ "a", "b", "c" ]
-    
-    {{ 42, "forty-two", "AsterixDB!" }}
-    
-    {
-      "project name"    : "AsterixDB"
-      "project members" : {{ "vinayakb", "dtabass", "chenli" }}
-    } 
-
-
-### Path Expressions
-
-    ValueExpr ::= PrimaryExpr ( Field | Index )*
-    Field     ::= "." Identifier
-    Index     ::= "[" ( Expression | "?" ) "]"
-
-##### Examples
-
-    { "list" : [ "a", "b", "c"] }.list
-    
-    [ "a", "b", "c"][2]
-    
-    { "list" : [ "a", "b", "c"] }.list[2]
-
-
-### Logical Expressions
-
-    OperatorExpr ::= AndExpr ( "or" AndExpr )*
-    AndExpr      ::= RelExpr ( "and" RelExpr )*
-    
-##### Example
-
-    $a > 3 and $a < 5
-    
-
-### Comparison Expressions
-
-    RelExpr ::= AddExpr ( ( "<" | ">" | "<=" | ">=" | "=" | "!=" | "~=" ) AddExpr )?
-    
-##### Example
-
-    5 > 3
-
-
-### Arithmetic Expressions
-
-    AddExpr  ::= MultExpr ( ( "+" | "-" ) MultExpr )*
-    MultExpr ::= UnaryExpr ( ( "*" | "/" | "%" | <CARET> | "idiv" ) UnaryExpr )*
-    UnaryExpr ::= ( ( "+" | "-" ) )? ValueExpr
-
-##### Example
-
-    3 ^ 2 + 4 ^ 2
-
-
-###  FLWOGR Expression   
-    
-    FLWOGR         ::= ( ForClause | LetClause ) ( Clause )* "return" Expression
-    Clause         ::= ForClause | LetClause | WhereClause | OrderbyClause
-                     | GroupClause | LimitClause | DistinctClause
-    ForClause      ::= "for" Variable ( "at" Variable )? "in" ( Expression )
-    LetClause      ::= "let" Variable ":=" Expression
-    WhereClause    ::= "where" Expression
-    OrderbyClause  ::= "order" "by" Expression ( ( "asc" ) | ( "desc" ) )? 
-                       ( "," Expression ( ( "asc" ) | ( "desc" ) )? )*
-    GroupClause    ::= "group" "by" ( Variable ":=" )? Expression ( "," ( Variable ":=" )? Expression )*          
-                       "with" VariableRef ( "," VariableRef )*
-    LimitClause    ::= "limit" Expression ( "offset" Expression )?
-    DistinctClause ::= "distinct" "by" Expression ( "," Expression )*
-    Variable       ::= <VARIABLE>
-
-
-##### Example
-
-    for $user in dataset FacebookUsers
-    where $user.id = 8
-    return $user
-    
-##### Example
-
-    for $user in dataset FacebookUsers
-    for $message in dataset FacebookMessages
-    where $message.author-id = $user.id
-    return
-      {
-        "uname": $user.name,
-        "message": $message.message
-      }; 
-    
-##### Example
-
-    for $user in dataset FacebookUsers
-    let $messages := 
-      for $message in dataset FacebookMessages
-      where $message.author-id = $user.id
-      return $message.message
-    return
-      {
-        "uname": $user.name,
-        "messages": $messages
-      }; 
-      
-##### Example
-      
-      for $user in dataset TwitterUsers
-      order by $user.followers_count desc, $user.lang asc
-      return $user
-      
-* null is smaller than any other value
-
-##### Example
-
-      for $x in dataset FacebookMessages
-      let $messages := $x.message
-      group by $loc := $x.sender-location with $messages
-      return
-        {
-          "location" : $loc,
-          "message" : $messages
-        }
-
-* after group by only variables that are either in the group-by-list or in the with-list are in scope
-* the variables in the with-clause contain a collection of items after the group by clause  (all the values that the variable was bound to in the tuples that make up the group)
-* null is handled as a single value for grouping
-
-##### Example
-
-      for $user in dataset TwitterUsers
-      order by $user.followers_count desc
-      limit 2
-      return $user
-
-##### Example (currently not working)
-    
-      for $x in dataset FacebookMessages
-      distinct by $x.sender-location
-      return
-        {
-          "location" : $x.sender-location,
-          "message" : $x.message
-        }
-
-* every variable that is in-scope before the distinct clause is also in scope after the distinct clause
-* works a lot like group by, but for every variable that contains more than one value after the distinct-by clause, one value is picked non-deterministically
-* if the variable is in the disctict-by list, then value is deterministic
-* null is a single value
-    
-### Conditional Expression
-    
-    IfThenElse ::= "if" "(" Expression ")" "then" Expression "else" Expression
-
-##### Example
-
-    if (2 < 3) then "yes" else "no"
-
-
-### Quantified Expressions
-    
-    QuantifiedExpression ::= ( ( "some" ) | ( "every" ) ) Variable "in" Expression 
-                             ( "," Variable "in" Expression )* "satisfies" Expression
-                             
-##### Examples
-
-    every $x in [ 1, 2, 3] satisfies $x < 3
-    some $x in [ 1, 2, 3] satisfies $x < 3
-
-## 3. Statements
-
-    Statement ::= ( SingleStatement ( ";" )? )* <EOF>
-    SingleStatement ::= DataverseDeclaration
-                      | FunctionDeclaration
-                      | CreateStatement
-                      | DropStatement
-                      | LoadStatement
-                      | SetStatement
-                      | InsertStatement
-                      | DeleteStatement
-                      | Query
-    
-### Declarations    
-    
-    DataverseDeclaration ::= "use" "dataverse" Identifier
-    SetStatement         ::= "set" Identifier StringLiteral
-    FunctionDeclaration  ::= "declare" "function" Identifier ParameterList "{" Expression "}"
-    ParameterList        ::= "(" ( <VARIABLE> ( "," <VARIABLE> )* )? ")"
-
-##### Example
-
-    use dataverse TinySocial;
-    
-##### Example
-
-    set simfunction "jaccard";
-    set simthreshold "0.6f"; 
-
-##### Example
-
-    set simfunction "jaccard";    
-    set simthreshold "0.6f"; 
-    
-##### Example
-    
-    declare function add($a, $b) {
-      $a + $b
-    };
-
-### Lifecycle Management Statements
-
-    CreateStatement ::= "create" ( DataverseSpecification
-                                 | TypeSpecification
-                                 | DatasetSpecification
-                                 | IndexSpecification
-                                 | FunctionSpecification )
-
-    QualifiedName       ::= Identifier ( "." Identifier )?
-    DoubleQualifiedName ::= Identifier "." Identifier ( "." Identifier )?
-
-#### Dataverses
-
-    DataverseSpecification ::= "dataverse" Identifier IfNotExists ( "with format" StringLiteral )?
-    
-
-##### Example
-
-    create dataverse TinySocial;
-
-#### Types
-
-    TypeSpecification    ::= "type" FunctionOrTypeName IfNotExists "as" TypeExpr
-    FunctionOrTypeName   ::= QualifiedName
-    IfNotExists          ::= ( "if not exists" )?
-    TypeExpr             ::= RecordTypeDef | TypeReference | OrderedListTypeDef | UnorderedListTypeDef
-    RecordTypeDef        ::= ( "closed" | "open" )? "{" ( RecordField ( "," RecordField )* )? "}"
-    RecordField          ::= Identifier ":" ( TypeExpr ) ( "?" )?
-    TypeReference        ::= Identifier
-    OrderedListTypeDef   ::= "[" ( TypeExpr ) "]"
-    UnorderedListTypeDef ::= "{{" ( TypeExpr ) "}}"
-
-##### Example
-
-    create type FacebookUserType as closed {
-      id: int32,
-      alias: string,
-      name: string,
-      user-since: datetime,
-      friend-ids: {{ int32 }},
-      employment: [EmploymentType]
-    }
-
-
-#### Datasets
-
-    DatasetSpecification ::= "internal"? "dataset" QualifiedName "(" Identifier ")" IfNotExists
-                             PrimaryKey ( "on" Identifier )? ( "hints" Properties )? 
-                           | "external" "dataset" QualifiedName "(" Identifier ")" IfNotExists 
-                             "using" AdapterName Configuration ( "hints" Properties )?
-    AdapterName          ::= Identifier
-    Configuration        ::= "(" ( KeyValuePair ( "," KeyValuePair )* )? ")"
-    KeyValuePair         ::= "(" StringLiteral "=" StringLiteral ")"
-    Properties           ::= ( "(" Property ( "," Property )* ")" )?
-    Property             ::= Identifier "=" ( StringLiteral | <INTEGER_LITERAL> )
-    ApplyFunction        ::= "apply" "function" FunctionSignature
-    FunctionSignature    ::= FunctionOrTypeName "@" <INTEGER_LITERAL>
-    PrimaryKey           ::= "primary" "key" Identifier ( "," Identifier )*
-
-
-##### Example
-    create internal dataset FacebookUsers(FacebookUserType) primary key id;
-
-##### Example
-
-    create external dataset Lineitem(LineitemType) using localfs (
-      ("path"="127.0.0.1://SOURCE_PATH"),
-      ("format"="delimited-text"),
-      ("delimiter"="|"));
-      
-#### Indices
-
-    IndexSpecification ::= "index" Identifier IfNotExists "on" QualifiedName 
-                           "(" ( Identifier ) ( "," Identifier )* ")" ( "type" IndexType )?
-    IndexType          ::= "btree"
-                         | "rtree"
-                         | "keyword"
-                         | "fuzzy keyword"
-                         | "ngram" "(" <INTEGER_LITERAL> ")"
-                         | "fuzzy ngram" "(" <INTEGER_LITERAL> ")"
-
-##### Example
-
-    create index fbAuthorIdx on FacebookMessages(author-id) type btree;
-
-##### Example
-
-    create index fbSenderLocIndex on FacebookMessages(sender-location) type rtree;
-
-##### Example
-
-    create index fbMessageIdx on FacebookMessages(message) type keyword;
-
-
-#### Functions
-
-    FunctionSpecification ::= "function" FunctionOrTypeName IfNotExists ParameterList "{" Expression "}"
-    
-##### Example
-    
-    create function add($a, $b) {
-      $a + $b
-    };
-    
-
-#### Removal
-
-    DropStatement       ::= "drop" ( "dataverse" Identifier IfExists
-                                   | "type" FunctionOrTypeName IfExists
-                                   | "dataset" QualifiedName IfExists
-                                   | "index" DoubleQualifiedName IfExists
-                                   | "function" FunctionSignature IfExists )
-    IfExists            ::= ( "if" "exists" )?
-    
-##### Example
-
-    drop dataset FacebookUsers if exists;
-
-##### Example
-
-    drop index fbSenderLocIndex;
-
-##### Example
-
-    drop type FacebookUserType;
-    
-##### Example
-
-    drop dataverse TinySocial;
-
-##### Example
-
-    drop function add;
-    
-
-### Import/Export Statements
-
-    LoadStatement  ::= "load" "dataset" QualifiedName "using" AdapterName Configuration ( "pre-sorted" )?
-    
-##### Example
-
-    load dataset FacebookUsers using localfs
-    (("path"="localhost:///Users/zuck/AsterixDB/load/fbu.adm"),("format"="adm"));
-
-
-### Modification Statements
-
-    InsertStatement ::= "insert" "into" "dataset" QualifiedName Query
-    DeleteStatement ::= "delete" Variable "from" "dataset" QualifiedName ( "where" Expression )?
-    
-##### Example
-
-    insert into dataset UsersCopy (for $user in dataset FacebookUsers return $user)
-
-##### Example
-    
-    delete $user from dataset FacebookUsers where $user.id = 8;
-    
-
-### Queries
-
-    Query ::= Expression
-    
-##### Example
-    
-    for $praise in {{ "great", "brilliant", "awesome" }}
-    return
-       string-concat(["AsterixDB is ", $praise])
diff --git a/asterix-doc/src/site/markdown/AsterixSimilarityQueries.md b/asterix-doc/src/site/markdown/AsterixSimilarityQueries.md
deleted file mode 100644
index 4f22fef..0000000
--- a/asterix-doc/src/site/markdown/AsterixSimilarityQueries.md
+++ /dev/null
@@ -1,83 +0,0 @@
-# AsterixDB Support of Similarity Queries #
-
-## Motivation ##
-
-Similarity queries are widely used in applications where users need to find records that satisfy a similarity predicate, while exact matching is not sufficient. These queries are especially important for social and Web applications, where errors, abbreviations, and inconsistencies are common.  As an example, we may want to find all the movies starring Schwarzenegger, while we don't know the exact spelling of his last name (despite his popularity in both the movie industry and politics :-)). As another example, we want to find all the Facebook users who have similar friends. To meet this type of needs, AsterixDB supports similarity queries using efficient indexes and algorithms.
-
-## Data Types and Similarity Functions ##
-
-AsterixDB supports various similarity functions, including [edit distance](http://en.wikipedia.org/wiki/Levenshtein_distance) (on strings) and [Jaccard](http://en.wikipedia.org/wiki/Jaccard_index) (on sets). For instance, in our [TinySocial](AdmAql101.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB) example, the `friend-ids` of a Facebook user forms a set of friends, and we can define a similarity between two sets. We can also convert a string to a set of "q-grams" and define the Jaccard similarity between the two sets of two strings. The "q-grams" of a string are its substrings of length "q". For instance, the 3-grams of the string `schwarzenegger` are `sch`, `chw`, `hwa`, ..., `ger`.
-
-AsterixDB provides [tokenization functions](AsterixDataTypesAndFunctions.html#Tokenizing_Functions) to convert strings to sets, and the [similarity functions](AsterixDataTypesAndFunctions.html#Similarity_Functions).
-
-## Selection Queries ##
-
-The following [query](AsterixDataTypesAndFunctions.html#edit-distance) asks for all the Facebook users whose name is similar to `Suzanna Tilson`, i.e., their edit distance is at most 2.
-
-
-        use dataverse TinySocial;
-        
-        for $user in dataset('FacebookUsers')
-        let $ed := edit-distance($user.name, "Suzanna Tilson")
-        where $ed <= 2
-        return $user
-
-
-The following [query](AsterixDataTypesAndFunctions.html#similarity-jaccard) asks for all the Facebook users whose set of friend ids is similar to `[1,5,9]`, i.e., their Jaccard similarity is at least 0.6.
-
-
-        use dataverse TinySocial;
-        
-        for $user in dataset('FacebookUsers')
-        let $sim := similarity-jaccard($user.friend-ids, [1,5,9])
-        where $sim >= 0.6f
-        return $user
-
-
-AsterixDB allows a user to use a similarity operator `~=` to express a similarity condition by defining the similiarty function and threshold using "set" statements earlier. For instance, the above query can be equivalently written as:
-
-
-        use dataverse TinySocial;
-        
-        set simfunction "jaccard";
-        set simthreshold "0.6f";
-        
-        for $user in dataset('FacebookUsers')
-        where $user.friend-ids ~= [1,5,9]
-        return $user
-
-
-
-## Fuzzy Join Queries ##
-
-AsterixDB supports fuzzy joins between two data sets. The following [query](AdmAql101.html#Query_5_-_Fuzzy_Join) finds, for each Facebook user, all Twitter users with names "similar" to their name based on the edit distance.
-
-
-        use dataverse TinySocial;
-        
-        set simfunction "edit-distance";
-        set simthreshold "3";
-        
-        for $fbu in dataset FacebookUsers
-        return {
-            "id": $fbu.id,
-            "name": $fbu.name,
-            "similar-users": for $t in dataset TweetMessages
-                                let $tu := $t.user
-                                where $tu.name ~= $fbu.name
-                                return {
-                                "twitter-screenname": $tu.screen-name,
-                                "twitter-name": $tu.name
-                                }
-        };
-
-
-## Using Indexes ##
-
-AsterixDB uses inverted index to support similarity queries efficiently. For instance, the following query creates such an index on the `FacebookUser.name` attribute using an inverted index of 3-grams.  After the index is created, similarity queries with an edit distance condition on this attribute can be answered more efficiently.
-
-
-        use dataverse TinySocial;
-        
-        create index fbUserFuzzyIdx on FacebookUsers(name) type ngram(3);
-
diff --git a/asterix-doc/src/site/markdown/AsterixDBRestAPI.md b/asterix-doc/src/site/markdown/api.md
similarity index 100%
rename from asterix-doc/src/site/markdown/AsterixDBRestAPI.md
rename to asterix-doc/src/site/markdown/api.md
diff --git a/asterix-doc/src/site/markdown/AsterixDBDataModel.md b/asterix-doc/src/site/markdown/aql/datamodel.md
similarity index 82%
rename from asterix-doc/src/site/markdown/AsterixDBDataModel.md
rename to asterix-doc/src/site/markdown/aql/datamodel.md
index ffefd09..3e54d61 100644
--- a/asterix-doc/src/site/markdown/AsterixDBDataModel.md
+++ b/asterix-doc/src/site/markdown/aql/datamodel.md
@@ -1,13 +1,12 @@
 # Asterix Data Model (ADM) #
 
-# Basic data types #
 
-An instance of Asterix data model (ADM) can be a _primitive type_ (`Int32`, `Int64`, `String`, `Float`, `Double`, `Date`, `Time`, `Datetime`, etc. or `NULL`) or a _derived type_.
+An instance of Asterix data model (ADM) can be a _*primitive type*_ (`int32`, `int64`, `string`, `float`, `double`, `date`, `time`, `datetime`, etc. or `null`) or a _*derived type*_.
 
 ## Primitive Types ##
 
 ### Boolean ###
-`Boolean` data type can have one of the two values: _*true*_ or _*false*_.
+`boolean` data type can have one of the two values: _*true*_ or _*false*_.
 
  * Example:
 
@@ -25,10 +24,10 @@
 ### Int8 / Int16 / Int32 / Int64 ###
 Integer types using 8, 16, 32, or 64 bits. The ranges of these types are:
 
-- `Int8`: -127 to 127
-- `Int16`: -32767 to 32767
-- `Int32`: -2147483647 to 2147483647
-- `Int64`: -9223372036854775808 to 9223372036854775807
+- `int8`: -127 to 127
+- `int16`: -32767 to 32767
+- `int32`: -2147483647 to 2147483647
+- `int64`: -9223372036854775808 to 9223372036854775807
 
  * Example:
 
@@ -45,7 +44,7 @@
 
 
 ### Float ###
-`Float` represents approximate numeric data values using 4 bytes. The range of a float value can be from 2^(-149) to (2-2^(-23)·2^(127) for both positive and negative. Beyond these ranges will get `INF` or `-INF`.
+`float` represents approximate numeric data values using 4 bytes. The range of a float value can be from 2^(-149) to (2-2^(-23)·2^(127) for both positive and negative. Beyond these ranges will get `INF` or `-INF`.
 
  * Example:
 
@@ -62,7 +61,7 @@
 
 
 ### Double ###
-`Double` represents approximate numeric data values using 8 bytes. The range of a double value can be from (2^(-1022)) to (2-2^(-52))·2^(1023) for both positive and negative. Beyond these ranges will get `INF` or `-INF`.
+`double` represents approximate numeric data values using 8 bytes. The range of a double value can be from (2^(-1022)) to (2-2^(-52))·2^(1023) for both positive and negative. Beyond these ranges will get `INF` or `-INF`.
 
  * Example:
 
@@ -79,7 +78,7 @@
 
 
 ### String ###
-`String` represents a sequence of characters.
+`string` represents a sequence of characters.
 
  * Example:
 
@@ -94,7 +93,7 @@
 
 
 ### Point ###
-`Point` is the fundamental two-dimensional building block for spatial types. It consists of two `double` coordinates x and y.
+`point` is the fundamental two-dimensional building block for spatial types. It consists of two `double` coordinates x and y.
 
  * Example:
 
@@ -109,7 +108,7 @@
 
 
 ### Line ###
-`Line` consists of two points that represent the start and the end points of a line segment.
+`line` consists of two points that represent the start and the end points of a line segment.
 
  * Example:
 
@@ -124,7 +123,7 @@
 
 
 ### Rectangle ###
-`Rectangle` consists of two points that represent the _*bottom left*_ and _*upper right*_ corners of a rectangle.
+`rectangle` consists of two points that represent the _*bottom left*_ and _*upper right*_ corners of a rectangle.
 
  * Example:
 
@@ -139,7 +138,7 @@
 
 
 ### Circle ###
-`Circle` consists of one `point` that represents the center of the circle and a radius of type `double`.
+`circle` consists of one point that represents the center of the circle and a radius of type `double`.
 
  * Example:
 
@@ -154,7 +153,7 @@
 
 
 ### Polygon ###
-`Polygon` consists of _*n*_ points that represent the vertices of a _*simple closed*_ polygon.
+`polygon` consists of _*n*_ points that represent the vertices of a _*simple closed*_ polygon.
 
  * Example:
 
@@ -169,7 +168,7 @@
 
 
 ### Date ###
-`Date` represents a time point along the Gregorian calendar system specified by the year, month and day. ASTERIX supports the date from `-9999-01-01` to `9999-12-31`.
+`date` represents a time point along the Gregorian calendar system specified by the year, month and day. ASTERIX supports the date from `-9999-01-01` to `9999-12-31`.
 
 A date value can be represented in two formats, extended format and basic format.
 
@@ -189,7 +188,7 @@
 
 
 ### Time ###
-`Time` type describes the time within the range of a day. It is represented by three fields: hour, minute and second. Millisecond field is optional as the fraction of the second field. Its extended format is as `hh:mm:ss[.mmm]` and the basic format is `hhmmss[mmm]`. The value domain is from `00:00:00.000` to `23:59:59.999`.
+`time` type describes the time within the range of a day. It is represented by three fields: hour, minute and second. Millisecond field is optional as the fraction of the second field. Its extended format is as `hh:mm:ss[.mmm]` and the basic format is `hhmmss[mmm]`. The value domain is from `00:00:00.000` to `23:59:59.999`.
 
 Timezone field is optional for a time value. Timezone is represented as `[+|-]hh:mm` for extended format or `[+|-]hhmm` for basic format. Note that the sign designators cannot be omitted. `Z` can also be used to represent the UTC local time. If no timezone information is given, it is UTC by default.
 
@@ -206,11 +205,11 @@
 
 
 ### Datetime ###
-A `Datetime` value is a combination of an `Date` and `Time`, representing a fixed time point along the Gregorian calendar system. The value is among `-9999-01-01 00:00:00.000` and `9999-12-31 23:59:59.999`.
+A `datetime` value is a combination of an `date` and `time`, representing a fixed time point along the Gregorian calendar system. The value is among `-9999-01-01 00:00:00.000` and `9999-12-31 23:59:59.999`.
 
-A `Datetime` value is represented as a combination of the representation of its `Date` part and `Time` part, separated by a separator `T`. Either extended or basic format can be used, and the two parts should be the same format.
+A `datetime` value is represented as a combination of the representation of its `date` part and `time` part, separated by a separator `T`. Either extended or basic format can be used, and the two parts should be the same format.
 
-Millisecond field and timezone field are optional, as specified in the `Time` type.
+Millisecond field and timezone field are optional, as specified in the `time` type.
 
  * Example:
 
@@ -225,11 +224,11 @@
 
 
 ### Duration ###
-`Duration` represents a duration of time. A duration value is specified by integers on at least one of the following fields: year, month, day, hour, minute, second, and millisecond.
+`duration` represents a duration of time. A duration value is specified by integers on at least one of the following fields: year, month, day, hour, minute, second, and millisecond.
 
 A duration value is in the format of `[-]PnYnMnDTnHnMn.mmmS`. The millisecond part (as the fraction of the second field) is optional, and when no millisecond field is used, the decimal point should also be absent.
 
-Negative durations are also supported for the arithmetic operations between time instance types (`Date`, `Time` and `Datetime`), and is used to roll the time back for the given duration. For example `date("2012-01-01") + duration("-P3D")` will return `date("2011-12-29")`.
+Negative durations are also supported for the arithmetic operations between time instance types (`date`, `time` and `datetime`), and is used to roll the time back for the given duration. For example `date("2012-01-01") + duration("-P3D")` will return `date("2011-12-29")`.
 
 Note that a canonical representation of the duration is always returned, regardless whether the duration is in the canonical representation or not from the user's input. More information about canonical representation can be found from [XPath dayTimeDuration Canonical Representation](http://www.w3.org/TR/xpath-functions/#canonical-dayTimeDuration) and [yearMonthDuration Canonical Representation](http://www.w3.org/TR/xpath-functions/#canonical-yearMonthDuration).
 
@@ -246,7 +245,7 @@
 
 
 ### Interval ###
-`Interval` represents inclusive-exclusive ranges of time. It is defined by two time point values with the same temporal type(`Date`, `Time` or `Datetime`).
+`interval` represents inclusive-exclusive ranges of time. It is defined by two time point values with the same temporal type(`date`, `time` or `datetime`).
 
  * Example:
 
@@ -264,7 +263,7 @@
 ## Derived Types ##
 
 ### Record ###
-A `Record` contains a set of fields, where each field is described by its name and type. A record type is either open or closed. Open records can contain fields that are not part of the type definition, while closed records cannot. Syntactically, record constructors are surrounded by curly braces "{...}".
+A `record` contains a set of fields, where each field is described by its name and type. A record type is either open or closed. Open records can contain fields that are not part of the type definition, while closed records cannot. Syntactically, record constructors are surrounded by curly braces "{...}".
 
 An example would be
 
@@ -273,7 +272,7 @@
 
 
 ### OrderedList ###
-An `OrderedList` is a sequence of values for which the order is determined by creation or insertion. OrderedList constructors are denoted by brackets: "[...]".
+An `orderedList` is a sequence of values for which the order is determined by creation or insertion. OrderedList constructors are denoted by brackets: "[...]".
 
 An example would be
 
@@ -282,7 +281,7 @@
 
 
 ### UnorderedList ###
-An `UnorderedList` is an unordered sequence of values, similar to bags in SQL. UnorderedList constructors are denoted by two opening flower braces followed by data and two closing flower braces, like "{{...}}".
+An `unorderedList` is an unordered sequence of values, similar to bags in SQL. UnorderedList constructors are denoted by two opening flower braces followed by data and two closing flower braces, like "{{...}}".
 
 An example would be
 
diff --git a/asterix-doc/src/site/markdown/AccessingExternalDataInAsterixDB.md b/asterix-doc/src/site/markdown/aql/externaldata.md
similarity index 97%
rename from asterix-doc/src/site/markdown/AccessingExternalDataInAsterixDB.md
rename to asterix-doc/src/site/markdown/aql/externaldata.md
index 7e49a0f..e603954 100644
--- a/asterix-doc/src/site/markdown/AccessingExternalDataInAsterixDB.md
+++ b/asterix-doc/src/site/markdown/aql/externaldata.md
@@ -10,11 +10,11 @@
 
 As an example we consider the Lineitem dataset from [TPCH schema](http://www.openlinksw.com/dataspace/doc/dav/wiki/Main/VOSTPCHLinkedData/tpch.sql).
 
-We assume that you have successfully created an ASTERIX instance following the instructions at [Installing Asterix Using Managix](InstallingAsterixUsingManagix.html).
+We assume that you have successfully created an ASTERIX instance following the instructions at [Installing Asterix Using Managix](../install.html).
 _For constructing an example, we assume a single machine setup._
 
 Similar to a regular dataset, an external dataset has an associated datatype.  We shall first create the datatype associated with each record in Lineitem data.
-Paste the following in the query textbox on the webpage at http://127.0.0.1 and hit 'Execute'.
+Paste the following in the query textbox on the webpage at http://127.0.0.1:19001 and hit 'Execute'.
 
 
         create dataverse ExternalFileDemo;
@@ -100,7 +100,7 @@
         127.0.0.1:///home/joe/lineitem.tbl.
 
 
-In your web-browser, navigate to 127.0.0.1 and paste the above to the query text box. Finally hit 'Execute'.
+In your web-browser, navigate to 127.0.0.1:19001 and paste the above to the query text box. Finally hit 'Execute'.
 
 Next we move over to the the section [Writing Queries against an External Dataset](#Writing_Queries_against_an_External_Dataset) and try a sample query against the external dataset.
 
diff --git a/asterix-doc/src/site/markdown/AsterixDBFunctions.md b/asterix-doc/src/site/markdown/aql/functions.md
similarity index 80%
rename from asterix-doc/src/site/markdown/AsterixDBFunctions.md
rename to asterix-doc/src/site/markdown/aql/functions.md
index d47752c..1f46fca 100644
--- a/asterix-doc/src/site/markdown/AsterixDBFunctions.md
+++ b/asterix-doc/src/site/markdown/aql/functions.md
@@ -9,7 +9,7 @@
 
  * Converts the string `string_expression` to its code-based representation.
  * Arguments:
-   * `string_expression` : A `String` that will be converted.
+   * `string_expression` : A `string` that will be converted.
  * Return Value:
    * An `OrderedList` of the code points for the string `string_expression`.
 
@@ -22,7 +22,7 @@
  * Arguments:
    * `list_expression` : An `OrderedList` of code-points.
  * Return Value:
-   * A `String` representation of `list_expression`.
+   * A `string` representation of `list_expression`.
 
  * Example:
 
@@ -42,14 +42,14 @@
 ### contains ###
  * Syntax:
 
-        contains(string_expression, string_pattern)
+        contains(string_expression, substring_to_contain)
 
- * Checks whether the string `string_expression` contains the string `string_pattern`
+ * Checks whether the string `string_expression` contains the string `substring_to_contain`
  * Arguments:
-   * `string_expression` : A `String` that might contain the pattern.
-   * `string_pattern` : A target `String` that might be contained.
+   * `string_expression` : A `string` that might contain the given substring.
+   * `substring_to_contain` : A target `string` that might be contained.
  * Return Value:
-   * A `Boolean`, returns `true` if `string_expression` contains `string_pattern`, otherwise returns `false`.
+   * A `boolean`, returns `true` if `string_expression` contains `substring_to_contain`, otherwise returns `false`.
 
  * Example:
 
@@ -74,9 +74,9 @@
 
  * Returns the length of the list `list_expression`.
  * Arguments:
-   * `list_expression` : An `OrderedList`, `UnorderedList` or `NULL`, represents the list need to be checked.
+   * `list_expression` : An `OrderedList`, `UnorderedList` or `null`, represents the list need to be checked.
  * Return Value:
-   * An `Int32` that represents the length of `list_expression`.
+   * An `int32` that represents the length of `list_expression`.
 
  * Example:
 
@@ -96,12 +96,12 @@
 
         like(string_expression, string_pattern)
 
- * Checks whether the string `string_expression` contains the string `string_pattern`.
+ * Checks whether the string `string_expression` contains the string pattern `string_pattern`. Compared with `contains` function, `like` function also supports regex keywords.
  * Arguments:
-   * `string_expression` : A `String` that might contain the pattern or `NULL`.
-   * `string_pattern` : A pattern `String` that might be contained or `NULL`.
+   * `string_expression` : A `string` that might contain the pattern or `null`.
+   * `string_pattern` : A pattern `string` that might be contained or `null`.
  * Return Value:
-   * A `Boolean`, returns `true` if `string_expression` contains the pattern `string_pattern`, otherwise returns `false`.
+   * A `boolean`, returns `true` if `string_expression` contains the pattern `string_pattern`, otherwise returns `false`.
 
  * Example:
 
@@ -122,14 +122,14 @@
 ### starts-with ###
  * Syntax:
 
-        starts-with(string_expression, string_pattern)
+        starts-with(string_expression, substring_to_start_with)
 
- * Checks whether the string `string_expression` starts with the string `string_pattern`.
+ * Checks whether the string `string_expression` starts with the string `substring_to_start_with`.
  * Arguments:
-   * `string_expression` : A `String` that might start with the given string.
-   * `string_pattern` : A `String` that might be contained as the starting substring.
+   * `string_expression` : A `string` that might start with the given string.
+   * `substring_to_start_with` : A `string` that might be contained as the starting substring.
  * Return Value:
-   * A `Boolean`, returns `true` if `string_expression` starts with the string `string_pattern`, otherwise returns `false`.
+   * A `boolean`, returns `true` if `string_expression` starts with the string `substring_to_start_with`, otherwise returns `false`.
 
  * Example:
 
@@ -151,14 +151,14 @@
 ### ends-with ###
  * Syntax:
 
-        ends-with(string_expression, string_pattern)
+        ends-with(string_expression, substring_to_end_with)
 
- * Checks whether the string `string_expression` ends with the string `string_pattern`.
+ * Checks whether the string `string_expression` ends with the string `substring_to_end_with`.
  * Arguments:
-   * `string_expression` : A `String` that might end with the given string.
-   * `string_pattern` : A `String` that might be contained as the ending substring.
+   * `string_expression` : A `string` that might end with the given string.
+   * `substring_to_end_with` : A `string` that might be contained as the ending substring.
  * Return Value:
-   * A `Boolean`, returns `true` if `string_expression` ends with the string `string_pattern`, otherwise returns `false`.
+   * A `boolean`, returns `true` if `string_expression` ends with the string `substring_to_end_with`, otherwise returns `false`.
 
  * Example:
 
@@ -183,9 +183,9 @@
 
  * Concatenates a list of strings `list_expression` into a single string.
  * Arguments:
-   * `list_expression` : An `OrderedList` or `UnorderedList` of `String`s (could be `NULL`) to be concatenated.
+   * `list_expression` : An `OrderedList` or `UnorderedList` of `string`s (could be `null`) to be concatenated.
  * Return Value:
-   * Returns the concatenated `String` value.
+   * Returns the concatenated `string` value.
 
  * Example:
 
@@ -207,10 +207,10 @@
 
  * Checks whether the strings `string_expression1` and `string_expression2` are equal.
  * Arguments:
-   * `string_expression1` : A `String` to be compared.
-   * `string_expression2` : A `String` to be compared with.
+   * `string_expression1` : A `string` to be compared.
+   * `string_expression2` : A `string` to be compared with.
  * Return Value:
-   * A `Boolean`, returns `true` if `string_expression1` and `string_expression2` are equal, otherwise returns `false`.
+   * A `boolean`, returns `true` if `string_expression1` and `string_expression2` are equal, otherwise returns `false`.
 
  * Example:
 
@@ -232,8 +232,8 @@
 
  * Joins a list of strings `list_expression` with the given separator `string_expression` into a single string.
  * Arguments:
-   * `list_expression` : An `OrderedList` or `UnorderedList` of `String`s (could be NULL) to be joined.
-   * `string_expression` : A separator `String` value.
+   * `list_expression` : An `OrderedList` or `UnorderedList` of `string`s (could be `null`) to be joined.
+   * `string_expression` : A `string` as the separator.
  * Return Value:
    * Returns the joined `String`.
 
@@ -255,11 +255,11 @@
 
         lowercase(string_expression)
 
- * Returns the lowercase of a given string `string_expression`.
+ * Converts a given string `string_expression` to its lowercase form.
  * Arguments:
-   * `string_expression` : A `String` to be lowercased.
+   * `string_expression` : A `string` to be converted.
  * Return Value:
-   * Returns the lowercased `String`.
+   * Returns a `string` as the lowercase form of the given `string_expression`.
 
  * Example:
 
@@ -281,10 +281,10 @@
 
  * Checks whether the strings `string_expression` matches the given pattern `string_pattern`.
  * Arguments:
-   * `string_expression` : A `String` that might contain the pattern.
-   * `string_pattern` : A pattern `String` to be matched.
+   * `string_expression` : A `string` that might contain the pattern.
+   * `string_pattern` : A pattern `string` to be matched.
  * Return Value:
-   * A `Boolean`, returns `true` if `string_expression` matches the pattern `string_pattern`, otherwise returns `false`.
+   * A `boolean`, returns `true` if `string_expression` matches the pattern `string_pattern`, otherwise returns `false`.
 
  * Example:
 
@@ -308,11 +308,11 @@
 
  * Checks whether the strings `string_expression` matches the given pattern `string_pattern`, and replace the matched pattern `string_pattern` with the new pattern `string_replacement`.
  * Arguments:
-   * `string_expression` : A `String` that might contain the pattern.
-   * `string_pattern` : A pattern `String` to be matched.
-   * `string_replacement` : A pattern `String` to be used as the replacement.
+   * `string_expression` : A `string` that might contain the pattern.
+   * `string_pattern` : A pattern `string` to be matched.
+   * `string_replacement` : A pattern `string` to be used as the replacement.
  * Return Value:
-   * Returns a `String` that is obtained after the replacements.
+   * Returns a `string` that is obtained after the replacements.
 
  * Example:
 
@@ -335,9 +335,9 @@
 
  * Returns the length of the string `string_expression`.
  * Arguments:
-   * `string_expression` : A `String` or `NULL`, represents the string to be checked.
+   * `string_expression` : A `string` or `null`, represents the string to be checked.
  * Return Value:
-   * An `Int32` that represents the length of `string_expression`.
+   * An `int32` that represents the length of `string_expression`.
 
  * Example:
 
@@ -369,15 +369,15 @@
 ### substring ###
  * Syntax:
 
-        substring(string_expression, offset, length)
+        substring(string_expression, offset[, length])
 
- * Returns the substring from the given string `string_expression` based on the given start offset `offset`.
+ * Returns the substring from the given string `string_expression` based on the given start offset `offset` with the optional `length`.
  * Arguments:
-   * `string_expression` : A `String` as the string to be extracted.
-   * `offset` : An `Int32` as the starting offset of the substring in `string_expression`.
-   * `length` : (Optional) An `Int32` as the length of the substring.
+   * `string_expression` : A `string` as the string to be extracted.
+   * `offset` : An `int32` as the starting offset of the substring in `string_expression`.
+   * `length` : (Optional) An `int32` as the length of the substring.
  * Return Value:
-   * A `String` that represents the substring.
+   * A `string` that represents the substring.
 
  * Example:
 
@@ -400,10 +400,10 @@
 
  * Returns the substring from the given string `string_expression` before the given pattern `string_pattern`.
  * Arguments:
-   * `string_expression` : A `String` as the string to be extracted.
-   * `string_pattern` : A `String` as the string pattern to be searched.
+   * `string_expression` : A `string` as the string to be extracted.
+   * `string_pattern` : A `string` as the string pattern to be searched.
  * Return Value:
-   * A `String` that represents the substring.
+   * A `string` that represents the substring.
 
  * Example:
 
@@ -428,10 +428,10 @@
 
  * Returns the substring from the given string `string_expression` after the given pattern `string_pattern`.
  * Arguments:
-   * `string_expression` : A `String` as the string to be extracted.
-   * `string_pattern` : A `String` as the string pattern to be searched.
+   * `string_expression` : A `string` as the string to be extracted.
+   * `string_pattern` : A `string` as the string pattern to be searched.
  * Return Value:
-   * A `String` that represents the substring.
+   * A `string` that represents the substring.
 
  * Example:
 
@@ -455,12 +455,12 @@
 
         create-point(latitude, longitude)
 
- * Creates the primitive type `Point` using `latitude` and `longitude`.
+ * Creates the primitive type `point` using `latitude` and `longitude`.
  * Arguments:
-   * `latitude` : A `Double` that represents the latitude.
-   * `longitude` : A `Double` that represents the longitude.
+   * `latitude` : A `double` that represents the latitude.
+   * `longitude` : A `double` that represents the longitude.
  * Return Value:
-   * A `Point`, represents a spatial point created using the latitude and longitude provided in `latitude` and `longitude`.
+   * A `point`, represents a spatial point created using the latitude and longitude provided in `latitude` and `longitude`.
 
  * Example:
 
@@ -480,12 +480,12 @@
 
         create-line(point_expression1, point_expression2)
 
- * Creates the primitive type `Line` using `point_expression1` and `point_expression2`.
+ * Creates the primitive type `line` using `point_expression1` and `point_expression2`.
  * Arguments:
-   * `point_expression1` : A `Point` that represents the start point of the line.
-   * `point_expression2` : A `Point` that represents the end point of the line.
+   * `point_expression1` : A `point` that represents the start point of the line.
+   * `point_expression2` : A `point` that represents the end point of the line.
  * Return Value:
-   * A `Line`, represents a spatial line created using the points provided in `point_expression1` and `point_expression2`.
+   * A `line`, represents a spatial line created using the points provided in `point_expression1` and `point_expression2`.
 
  * Example:
 
@@ -505,12 +505,12 @@
 
         create-rectangle(point_expression1, point_expression2)
 
- * Creates the primitive type Rectangle using `point_expression1` and `point_expression2`.
+ * Creates the primitive type `rectangle` using `point_expression1` and `point_expression2`.
  * Arguments:
-   * `point_expression1` : A `Point` that represents the lower-left point of the rectangle.
-   * `point_expression2` : A `Point` that represents the upper-right point of the rectangle.
+   * `point_expression1` : A `point` that represents the lower-left point of the rectangle.
+   * `point_expression2` : A `point` that represents the upper-right point of the rectangle.
  * Return Value:
-   * A `Rectangle`, represents a spatial rectangle created using the points provided in `point_expression1` and `point_expression2`.
+   * A `rectangle`, represents a spatial rectangle created using the points provided in `point_expression1` and `point_expression2`.
 
  * Example:
 
@@ -530,12 +530,12 @@
 
         create-circle(point_expression, radius)
 
- * Creates the primitive type `Circle` using `point_expression` and `radius`.
+ * Creates the primitive type `circle` using `point_expression` and `radius`.
  * Arguments:
-   * `point_expression` : A `Point` that represents the center of the circle.
-   * `radius` : A `Double` that represents the radius of the circle.
+   * `point_expression` : A `point` that represents the center of the circle.
+   * `radius` : A `double` that represents the radius of the circle.
  * Return Value:
-   * A `Circle`, represents a spatial circle created using the center point and the radius provided in `point_expression` and `radius`.
+   * A `circle`, represents a spatial circle created using the center point and the radius provided in `point_expression` and `radius`.
 
  * Example:
 
@@ -553,13 +553,13 @@
 ### create-polygon ###
  * Syntax:
 
-        create-polygon(point_expression1, point_expression2, ..., point_expressionn)
+        create-polygon(point_expression1, point_expression2, point_expression3, […, point_expressionn])
 
- * Creates the primitive type `Polygon` using unlimited number of arguments `point_expression1`, `point_expression2`, ..., `point_expressionn`.
+ * Creates the primitive type `polygon` using unlimited number of arguments `point_expression1`, `point_expression2`, ..., `point_expressionn`. Note that at least three points should be specified.
  * Arguments:
-   * `point_expression1`/.../`point_expressionn` : A `Point` that represents a vertex of the polygon.
+   * `point_expression1`/.../`point_expressionn` : A `point` that represents a vertex of the polygon.
  * Return Value:
-   * A `Polygon`, represents a spatial simple polygon created using the points provided in `point_expression1`, `point_expression2`, ..., `point_expressionn`.
+   * A `polygon`, represents a spatial simple polygon created using the points provided in `point_expression1`, `point_expression2`, ..., `point_expressionn`.
 
  * Example:
 
@@ -579,11 +579,11 @@
 
         point(string_expression)
 
- * Constructor function for `Point` type by parsing a point string `string_expression`
+ * Constructor function for `point` type by parsing a point string `string_expression`
  * Arguments:
-   * `string_expression` : The `String` value representing a point value.
+   * `string_expression` : The `string` value representing a point value.
  * Return Value:
-   * A `Point` value represented by the given string.
+   * A `point` value represented by the given string.
 
  * Example:
 
@@ -604,11 +604,11 @@
 
         line(string_expression)
 
- * Constructor function for `Line` type by parsing a line string `string_expression`
+ * Constructor function for `line` type by parsing a line string `string_expression`
  * Arguments:
-   * `string_expression` : The `String` value representing a line value.
+   * `string_expression` : The `string` value representing a line value.
  * Return Value:
-   * A `Line` value represented by the given string.
+   * A `line` value represented by the given string.
 
  * Example:
 
@@ -629,11 +629,11 @@
 
         rectangle(string_expression)
 
- * Constructor function for `Rectangle` type by parsing a rectangle string `string_expression`
+ * Constructor function for `rectangle` type by parsing a rectangle string `string_expression`
  * Arguments:
-   * `string_expression` : The `String` value representing a rectangle value.
+   * `string_expression` : The `string` value representing a rectangle value.
  * Return Value:
-   * A `Rectangle` value represented by the given string.
+   * A `rectangle` value represented by the given string.
 
  * Example:
 
@@ -654,11 +654,11 @@
 
         circle(string_expression)
 
- * Constructor function for `Circle` type by parsing a circle string `string_expression`
+ * Constructor function for `circle` type by parsing a circle string `string_expression`
  * Arguments:
-   * `string_expression` : The `String` value representing a circle value.
+   * `string_expression` : The `string` value representing a circle value.
  * Return Value:
-   * A `Circle` value represented by the given string.
+   * A `circle` value represented by the given string.
 
  * Example:
 
@@ -679,11 +679,11 @@
 
         polygon(string_expression)
 
- * Constructor function for `Polygon` type by parsing a polygon string `string_expression`
+ * Constructor function for `polygon` type by parsing a polygon string `string_expression`
  * Arguments:
-   * `string_expression` : The `String` value representing a polygon value.
+   * `string_expression` : The `string` value representing a polygon value.
  * Return Value:
-   * A `Polygon` value represented by the given string.
+   * A `polygon` value represented by the given string.
 
  * Example:
 
@@ -706,9 +706,9 @@
 
  * Returns the x or y coordinates of a point `point_expression`.
  * Arguments:
-   * `point_expression` : A `Point`.
+   * `point_expression` : A `point`.
  * Return Value:
-   * A `Double`, represents the x or y coordinates of the point `point_expression`.
+   * A `double`, represents the x or y coordinates of the point `point_expression`.
 
  * Example:
 
@@ -730,7 +730,7 @@
 
  * Returns an ordered list of the points forming the spatial object `spatial_expression`.
  * Arguments:
-   * `spatial_expression` : A `Point`, `Line`, `Rectangle`, `Circle`, or `Polygon`.
+   * `spatial_expression` : A `point`, `line`, `rectangle`, `circle`, or `polygon`.
  * Return Value:
    * An `OrderedList` of the points forming the spatial object `spatial_expression`.
 
@@ -759,9 +759,9 @@
 
  * Returns the center and the radius of a circle `circle_expression`.
  * Arguments:
-   * `circle_expression` : A `Circle`.
+   * `circle_expression` : A `circle`.
  * Return Value:
-   * A `Point` or `Double`, represent the center or radius of the circle `circle_expression`.
+   * A `point` or `double`, represent the center or radius of the circle `circle_expression`.
 
  * Example:
 
@@ -785,10 +785,10 @@
 
  * Returns the euclidean distance between `point_expression1` and `point_expression2`.
  * Arguments:
-   * `point_expression1` : A `Point`.
-   * `point_expression2` : A `Point`.
+   * `point_expression1` : A `point`.
+   * `point_expression2` : A `point`.
  * Return Value:
-   * A `Double`, represents the euclidean distance between `point_expression1` and `point_expression2`.
+   * A `double`, represents the euclidean distance between `point_expression1` and `point_expression2`.
 
  * Example:
 
@@ -823,9 +823,9 @@
 
  * Returns the spatial area of `spatial_2d_expression`.
  * Arguments:
-   * `spatial_2d_expression` : A `Rectangle`, `Circle`, or `Polygon`.
+   * `spatial_2d_expression` : A `rectangle`, `circle`, or `polygon`.
  * Return Value:
-   * A `Double`, represents the area of `spatial_2d_expression`.
+   * A `double`, represents the area of `spatial_2d_expression`.
 
  * Example:
 
@@ -848,10 +848,10 @@
 
  * Checks whether `@arg1` and `@arg2` spatially intersect each other.
  * Arguments:
-   * `spatial_expression1` : A `Point`, `Line`, `Rectangle`, `Circle`, or `Polygon`.
-   * `spatial_expression2` : A `Point`, `Line`, `Rectangle`, `Circle`, or `Polygon`.
+   * `spatial_expression1` : A `point`, `line`, `rectangle`, `circle`, or `polygon`.
+   * `spatial_expression2` : A `point`, `line`, `rectangle`, `circle`, or `polygon`.
  * Return Value:
-   * A `Boolean`, represents whether `spatial_expression1` and `spatial_expression2` spatially intersect each other.
+   * A `boolean`, represents whether `spatial_expression1` and `spatial_expression2` spatially intersect each other.
 
  * Example:
 
@@ -876,12 +876,12 @@
 
  * Returns the grid cell that `point_expression1` belongs to.
  * Arguments:
-   * `point_expression1` : A `Point`, represents the point of interest that its grid cell will be returned.
-   * `point_expression2` : A `Point`, represents the origin of the grid.
-   * `x_increment` : A `Double`, represents X increments.
-   * `y_increment` : A `Double`, represents Y increments.
+   * `point_expression1` : A `point`, represents the point of interest that its grid cell will be returned.
+   * `point_expression2` : A `point`, represents the origin of the grid.
+   * `x_increment` : A `double`, represents X increments.
+   * `y_increment` : A `double`, represents Y increments.
  * Return Value:
-   * A `Rectangle`, represents the grid cell that `point_expression1` belongs to.
+   * A `rectangle`, represents the grid cell that `point_expression1` belongs to.
 
  * Example:
 
@@ -920,10 +920,10 @@
 
  * Returns the [edit distance](http://en.wikipedia.org/wiki/Levenshtein_distance) of `expression1` and `expression2`.
  * Arguments:
-   * `expression1` : A `String` or a homogeneous `OrderedList` of a comparable item type.
+   * `expression1` : A `string` or a homogeneous `OrderedList` of a comparable item type.
    * `expression2` : The same type as `expression1`.
  * Return Value:
-   * An `Int32` that represents the edit-distance similarity of `expression1` and `expression2`.
+   * An `int32` that represents the edit-distance similarity between `expression1` and `expression2`.
 
  * Example:
 
@@ -951,13 +951,13 @@
  * Checks whether `expression1` and `expression2` have a [edit distance](http://en.wikipedia.org/wiki/Levenshtein_distance) `<= threshold`.  The “check” version of edit distance is faster than the "non-check" version because the former can detect whether two items satisfy a given similarity threshold using early-termination techniques, as opposed to computing their real distance. Although possible, it is not necessary for the user to write queries using the “check” versions explicitly, since a rewrite rule can perform an appropriate transformation from a “non-check” version to a “check” version.
 
  * Arguments:
-   * `expression1` : A `String` or a homogeneous `OrderedList` of a comparable item type.
+   * `expression1` : A `string` or a homogeneous `OrderedList` of a comparable item type.
    * `expression2` : The same type as `expression1`.
-   * `threshold` : An `Int32` that represents the distance threshold.
+   * `threshold` : An `int32` that represents the distance threshold.
  * Return Value:
    * An `OrderedList` with two items:
-     * The first item contains a `Boolean` value representing whether `expression1` and `expression2` are similar.
-     * The second item contains an `Int32` that represents the edit distance of `expression1` and `expression2` if it is `<= `threshold`, or 0 otherwise.
+     * The first item contains a `boolean` value representing whether `expression1` and `expression2` are similar.
+     * The second item contains an `int32` that represents the edit distance of `expression1` and `expression2` if it is `<= `threshold`, or 0 otherwise.
 
  * Example:
 
@@ -984,7 +984,7 @@
    * `list_expression1` : An `UnorderedList` or `OrderedList`.
    * `list_expression2` : An `UnorderedList` or `OrderedList`.
  * Return Value:
-   * A `Float` that represents the Jaccard similarity of `list_expression1` and `list_expression2`.
+   * A `float` that represents the Jaccard similarity of `list_expression1` and `list_expression2`.
 
  * Example:
 
@@ -1018,11 +1018,11 @@
  * Arguments:
    * `list_expression1` : An `UnorderedList` or `OrderedList`.
    * `list_expression2` : An `UnorderedList` or `OrderedList`.
-   * `threshold` : A Float that represents the similarity threshold.
+   * `threshold` : A `float` that represents the similarity threshold.
  * Return Value:
    * An `OrderedList` with two items:
-     * The first item contains a `Boolean` value representing whether `list_expression1` and `list_expression2` are similar.
-     * The second item contains a `Float` that represents the Jaccard similarity of `list_expression1` and `list_expression2` if it is >`= `threshold`, or 0 otherwise.
+     * The first item contains a `boolean` value representing whether `list_expression1` and `list_expression2` are similar.
+     * The second item contains a `float` that represents the Jaccard similarity of `list_expression1` and `list_expression2` if it is >`= `threshold`, or 0 otherwise.
 
  * Example:
 
@@ -1043,7 +1043,7 @@
 ### Similarity Operator ~# ###
  * "`~=`" is syntactic sugar for expressing a similarity condition with a given similarity threshold.
  * The similarity function and threshold for "`~=`" are controlled via "set" directives.
- * The "`~=`" operator returns a Boolean that represents whether the operands are similar.
+ * The "`~=`" operator returns a `boolean` that represents whether the operands are similar.
 
  * Example for Jaccard similarity:
 
@@ -1097,9 +1097,9 @@
 
  * Returns a list of word tokens of `string_expression`.
  * Arguments:
-   * `string_expression` : A `String` that will be tokenized.
+   * `string_expression` : A `string` that will be tokenized.
  * Return Value:
-   * An `OrderedList` of `String` word tokens.
+   * An `OrderedList` of `string` word tokens.
 
  * Example:
 
@@ -1119,16 +1119,16 @@
         { "tweetid": "9", "word-tokens": [ "love", "verizon", "its", "voicemail", "service", "is", "awesome" ] }
 
 
-### hashed-word-tokens ###
+<!--### hashed-word-tokens ###
  * Syntax:
 
         hashed-word-tokens(string_expression)
 
  * Returns a list of hashed word tokens of `string_expression`.
  * Arguments:
-   * `string_expression` : A `String` that will be tokenized.
+   * `string_expression` : A `string` that will be tokenized.
  * Return Value:
-   * An `OrderedList` of Int32 hashed tokens.
+   * An `OrderedList` of `int32` hashed tokens.
 
  * Example:
 
@@ -1283,7 +1283,7 @@
         -790946468, -69070309, 1561601454, 26169001, -160734571, 1330043462, -486734512, -18796768, -470303314, 113421364, 1615760212, 1688217556, 1223719184, 536568131, 1682609873, 2935161,
         -414769471, -1027490137, 1602276102, 1050490461 ]
         }
-
+-->
 
 ## Temporal Functions ##
 
@@ -1292,11 +1292,11 @@
 
         date(string_expression)
 
- * Constructor function for `Date` type by parsing a date string `string_expression`
+ * Constructor function for `date` type by parsing a date string `string_expression`
  * Arguments:
-   * `string_expression` : The `String` value representing a date value.
+   * `string_expression` : The `string` value representing a date value.
  * Return Value:
-   * A `Date` value represented by the given string.
+   * A `date` value represented by the given string.
 
  * Example:
 
@@ -1319,11 +1319,11 @@
 
         time(string_expression)
 
- * Constructor function for `Time` type by parsing a time string `string_expression`
+ * Constructor function for `time` type by parsing a time string `string_expression`
  * Arguments:
-   * `string_expression` : The `String` value representing a time value.
+   * `string_expression` : The `string` value representing a time value.
  * Return Value:
-   * A `Time` value represented by the given string.
+   * A `time` value represented by the given string.
 
  * Example:
 
@@ -1346,11 +1346,11 @@
 
         datetime(string_expression)
 
- * Constructor function for `Datetime` type by parsing a datetime string `string_expression`
+ * Constructor function for `datetime` type by parsing a datetime string `string_expression`
  * Arguments:
-   * `string_expression` : The `String` value representing a datetime value.
+   * `string_expression` : The `string` value representing a datetime value.
  * Return Value:
-   * A `Datetime` value represented by the given string.
+   * A `datetime` value represented by the given string.
 
  * Example:
 
@@ -1373,12 +1373,12 @@
 
         interval-from-date(string_expression1, string_expression2)
 
- * Constructor function for `Interval` type by parsing two date strings.
+ * Constructor function for `interval` type by parsing two date strings.
  * Arguments:
-   * `string_expression1` : The `String` value representing the starting date.
-   * `string_expression2` : The `String` value representing the ending date.
+   * `string_expression1` : The `string` value representing the starting date.
+   * `string_expression2` : The `string` value representing the ending date.
  * Return Value:
-   * An `Interval` value between the two dates.
+   * An `interval` value between the two dates.
 
  * Example:
 
@@ -1395,12 +1395,12 @@
 
         interval-from-time(string_expression1, string_expression2)
 
- * Constructor function for `Interval` type by parsing two time strings.
+ * Constructor function for `interval` type by parsing two time strings.
  * Arguments:
-   * `string_expression1` : The `String` value representing the starting time.
-   * `string_expression2` : The `String` value representing the ending time.
+   * `string_expression1` : The `string` value representing the starting time.
+   * `string_expression2` : The `string` value representing the ending time.
  * Return Value:
-   * An `Interval` value between the two times.
+   * An `interval` value between the two times.
 
  * Example:
 
@@ -1417,12 +1417,12 @@
 
         interval-from-datetime(string_expression1, string_expression2)
 
- * Constructor function for `Interval` type by parsing two datetime strings.
+ * Constructor function for `interval` type by parsing two datetime strings.
  * Arguments:
-   * `string_expression1` : The `String` value representing the starting datetime.
-   * `string_expression2` : The `String` value representing the ending datetime.
+   * `string_expression1` : The `string` value representing the starting datetime.
+   * `string_expression2` : The `string` value representing the ending datetime.
  * Return Value:
-   * An `Interval` value between the two datetimes.
+   * An `interval` value between the two datetimes.
 
  * Example:
 
@@ -1441,9 +1441,9 @@
 
  * Accessors for accessing fields in a temporal value
  * Arguments:
-   * `temporal_expression` : a temporal value represented as one of the following types: `Date`, `Datetime`, `Time`, `Duration`.
+   * `temporal_expression` : a temporal value represented as one of the following types: `date`, `datetime`, `time`, `duration`.
  * Return Value:
-   * An `Int32` value representing the field to be extracted.
+   * An `int32` value representing the field to be extracted.
 
  * Example:
 
@@ -1468,10 +1468,10 @@
 
  * Create a new date by adding the duration `duration_expression` to the given date `date_expression`.
  * Arguments:
-   * `date_expression` : The `Date` value to be added onto.
-   * `duration_expression` : The `Duration` to be added.
+   * `date_expression` : The `date` value to be added onto.
+   * `duration_expression` : The `duration` to be added.
  * Return Value:
-   * A `Date` value represents the new date after being adjusted by the duration.
+   * A `date` value represents the new date after being adjusted by the duration.
 
  * Example:
 
@@ -1498,10 +1498,10 @@
 
  * Create a new datetime by adding the duration `duration_expression` to the given datetime `datetime_expression`.
  * Arguments:
-   * `datetime_expression` : The `Datetime` value to be added onto.
-   * `duration_expression` : The `Duration` to be added.
+   * `datetime_expression` : The `datetime` value to be added onto.
+   * `duration_expression` : The `duration` to be added.
  * Return Value:
-   * A `Datetime` value represents the new datetime after being adjusted by the duration.
+   * A `datetime` value represents the new datetime after being adjusted by the duration.
 
  * Example:
 
@@ -1527,10 +1527,10 @@
 
  * Create a new time by adding the duration `duration_expression` to the given time `time_expression`.
  * Arguments:
-   * `time_expression` : The `Time` value to be added onto.
-   * `duration_expression` : The `Duration` to be added.
+   * `time_expression` : The `time` value to be added onto.
+   * `duration_expression` : The `duration` to be added.
  * Return Value:
-   * A `Time` value represents the new time after being adjusted by the duration.
+   * A `time` value represents the new time after being adjusted by the duration.
 
  * Example:
 
@@ -1565,10 +1565,10 @@
 
  * Adjust the given datetime `datetime_expression` by applying the timezone information `string_expression`
  * Arguments:
-   * `datetime_expression` : A `Datetime` value to be adjusted.
-   * `string_expression` : A `String` representing the timezone information.
+   * `datetime_expression` : A `datetime` value to be adjusted.
+   * `string_expression` : A `string` representing the timezone information.
  * Return Value:
-   * A `String` value represents the new datetime after being adjusted by the timezone information.
+   * A `string` value represents the new datetime after being adjusted by the timezone information.
 
  * Example:
 
@@ -1601,10 +1601,10 @@
 
  * Adjust the given time `time_expression` by applying the timezone information `string_expression`
  * Arguments:
-   * `time_expression` : A `Time` value to be adjusted.
-   * `string_expression` : A `String` representing the timezone information.
+   * `time_expression` : A `time` value to be adjusted.
+   * `string_expression` : A `string` representing the timezone information.
  * Return Value:
-   * A `String` value represents the new time after being adjusted by the timezone information.
+   * A `string` value represents the new time after being adjusted by the timezone information.
 
  * Example:
 
@@ -1637,10 +1637,10 @@
 
  * Get a user-friendly representation of the duration `duration_expression` based on the given datetime `datetime_expression`
  * Arguments:
-   * `datetime_expression` : A `Datetime` value to be used as the reference time point.
-   * `duration_expression` : A `Duration` value to be converted
+   * `datetime_expression` : A `datetime` value to be used as the reference time point.
+   * `duration_expression` : A `duration` value to be converted
  * Return Value:
-   * A `Duration` value with the duration as `duration_expression` but with a user-friendly representation.
+   * A `duration` value with the duration as `duration_expression` but with a user-friendly representation.
 
  * Example:
 
@@ -1665,10 +1665,10 @@
 
  * Get a user-friendly representation of the duration `duration_expression` based on the given date `date_expression`
  * Arguments:
-   * `date_expression` : A `Date` value to be used as the reference time point.
-   * `duration_expression` : A `Duration` value to be converted
+   * `date_expression` : A `date` value to be used as the reference time point.
+   * `duration_expression` : A `duration` value to be converted
  * Return Value:
-   * A `Duration` value with the duration as `duration_expression` but with a user-friendly representation.
+   * A `duration` value with the duration as `duration_expression` but with a user-friendly representation.
 
  * Example:
 
@@ -1695,7 +1695,7 @@
  * Get the current date
  * Arguments:None
  * Return Value:
-   * A `Date` value of the date when the function is called.
+   * A `date` value of the date when the function is called.
 
 ### current-time ###
  * Syntax:
@@ -1705,7 +1705,7 @@
  * Get the current time
  * Arguments:None
  * Return Value:
-   * A `Time` value of the time when the function is called.
+   * A `time` value of the time when the function is called.
 
 ### current-datetime ###
  * Syntax:
@@ -1715,7 +1715,7 @@
  * Get the current datetime
  * Arguments:None
  * Return Value:
-   * A `Datetime` value of the datetime when the function is called.
+   * A `datetime` value of the datetime when the function is called.
 
  * Example:
 
@@ -1740,9 +1740,9 @@
 
  * Get the date value from the given datetime value `datetime_expression`
  * Arguments:
-   * `datetime_expression`: A `Datetime` value to be extracted from
+   * `datetime_expression`: A `datetime` value to be extracted from
  * Return Value:
-   * A `Date` value from the datetime.
+   * A `date` value from the datetime.
 
 ### time-from-datetime ###
  * Syntax:
@@ -1751,9 +1751,9 @@
 
  * Get the time value from the given datetime value `datetime_expression`
  * Arguments:
-   * `datetime_expression`: A `Datetime` value to be extracted from
+   * `datetime_expression`: A `datetime` value to be extracted from
  * Return Value:
-   * A `Time` value from the datetime.
+   * A `time` value from the datetime.
 
  * Example:
 
@@ -1778,9 +1778,9 @@
 
  * Get date representing the time after `numeric_expression` days since 1970-01-01
  * Arguments:
-   * `numeric_expression`: A `Int8`/`Int16`/`Int32` value representing the number of days
+   * `numeric_expression`: A `int8`/`int16`/`int32` value representing the number of days
  * Return Value:
-   * A `Date` value as the time after `numeric_expression` days since 1970-01-01
+   * A `date` value as the time after `numeric_expression` days since 1970-01-01
 
 ### datetime-from-unix-time-in-ms ###
  * Syntax:
@@ -1789,9 +1789,9 @@
 
  * Get datetime representing the time after `numeric_expression` milliseconds since 1970-01-01T00:00:00Z
  * Arguments:
-   * `numeric_expression`: A `Int8`/`Int16`/`Int32`/`Int64` value representing the number of milliseconds
+   * `numeric_expression`: A `int8`/`int16`/`int32`/`int64` value representing the number of milliseconds
  * Return Value:
-   * A `Datetime` value as the time after `numeric_expression` milliseconds since 1970-01-01T00:00:00Z
+   * A `datetime` value as the time after `numeric_expression` milliseconds since 1970-01-01T00:00:00Z
 
 ### time-from-unix-time-in-ms ###
  * Syntax:
@@ -1800,9 +1800,9 @@
 
  * Get time representing the time after `numeric_expression` milliseconds since 00:00:00.000Z
  * Arguments:
-   * `numeric_expression`: A `Int8`/`Int16`/`Int32` value representing the number of milliseconds
+   * `numeric_expression`: A `int8`/`int16`/`int32` value representing the number of milliseconds
  * Return Value:
-   * A `Time` value as the time after `numeric_expression` milliseconds since 00:00:00.000Z
+   * A `time` value as the time after `numeric_expression` milliseconds since 00:00:00.000Z
 
  * Example:
 
@@ -1826,10 +1826,10 @@
 
  * Get the duration between two dates `date_start` and `date_end`
  * Arguments:
-   * `date_start`: the starting `Date`
-   * `date_end`: the ending `Date`
+   * `date_start`: the starting `date`
+   * `date_end`: the ending `date`
  * Return Value:
-   * A `Duration` value between `date_start` and `date_end`
+   * A `duration` value between `date_start` and `date_end`
 
  * Example:
 
@@ -1855,10 +1855,10 @@
 
  * Get the duration between two times `time_start` and `time_end`
  * Arguments:
-   * `time_start`: the starting `Time`
-   * `time_end`: the ending `Time`
+   * `time_start`: the starting `time`
+   * `time_end`: the ending `time`
  * Return Value:
-   * A `Duration` value between `time_start` and `time_end`
+   * A `duration` value between `time_start` and `time_end`
 
  * Example:
 
@@ -1884,10 +1884,10 @@
 
  * Get the duration between two datetimes `datetime_start` and `datetime_end`
  * Arguments:
-   * `datetime_start`: the starting `Datetime`
-   * `datetime_end`: the ending `Datetime`
+   * `datetime_start`: the starting `datetime`
+   * `datetime_end`: the ending `datetime`
  * Return Value:
-   * A `Duration` value between `datetime_start` and `datetime_end`
+   * A `duration` value between `datetime_start` and `datetime_end`
 
  * Example:
 
@@ -1908,3 +1908,23 @@
         { "id1": 3, "id2": 7, "diff": duration("P28D") }
         { "id1": 7, "id2": 1, "diff": duration("P13D") }
 
+### get-interval-start, get-interval-end ###
+ * Syntax:
+
+        get-interval-start/get-interval-end(interval)
+
+ * Get the start/end of the given interval
+ * Arguments:
+   * `interval`: the interval to be accessed
+ * Return Value:
+   * A `time`, `date` or `datetime` (depending on the time instances of the interval) representing the starting or ending time.
+
+ * Example:
+
+        let $itv := interval-start-from-date("1984-01-01", "P1Y")
+        return {"start": get-interval-start($itv), "end": get-interval-end($itv)}
+
+
+ * The expected result is:
+
+        { "start": date("1984-01-01"), "end": date("1985-01-01") }
\ No newline at end of file
diff --git a/asterix-doc/src/site/markdown/aql/manual.md b/asterix-doc/src/site/markdown/aql/manual.md
new file mode 100644
index 0000000..f1c3fbd
--- /dev/null
+++ b/asterix-doc/src/site/markdown/aql/manual.md
@@ -0,0 +1,734 @@
+# The Asterix Query Language, Version 1.0
+## 1. Introduction
+
+This document is intended as a reference guide to the full syntax
+and semantics of the Asterix Query Language (AQL), the language for talking to AsterixDB.
+This guide covers both the data manipulation language (DML) aspects of AQL, including
+its support for queries and data modification, as well as its data definition language
+(DDL) aspects.
+ New AsterixDB users are encouraged to read and work through the (friendlier) guide
+"AsterixDB 101: An ADM and AQL Primer" before attempting to make use of this document.
+In addition, readers are advised to read and understand the Asterix Data Model (ADM)
+reference guide since a basic understanding of ADM concepts is a prerequisite to understanding AQL.  
+In what follows, we detail the features of the AQL language in a grammar-guided manner:
+We list and briefly explain each of the productions in the AQL grammar, offering 
+examples for clarity in cases where doing so seems needed or helpful.
+
+## 2. Expressions
+
+    Query ::= Expression
+
+An AQL query can be any legal AQL expression.
+    
+    Expression ::= ( OperatorExpr | IfThenElse | FLWOR | QuantifiedExpression )
+
+AQL is a fully composable expression language.
+Each AQL expression returns zero or more Asterix Data Model (ADM) instances.
+There are four major kinds of expressions in AQL.
+At the topmost level, an AQL expression can be an
+OperatorExpr (similar to a mathematical expression),
+an IfThenElse (to choose between two alternative values),
+a FLWOR expression (the heart of AQL, pronounced "flower expression"),
+or a QuantifiedExpression (which yields a boolean value).
+Each will be detailed as we explore the full AQL grammar.
+
+### Primary Expressions
+
+    PrimaryExpr ::= Literal
+                  | VariableRef
+                  | ParenthesizedExpression
+                  | FunctionCallExpr
+                  | DatasetAccessExpression
+                  | ListConstructor
+                  | RecordConstructor
+
+The most basic building block for any AQL expression is the PrimaryExpr.
+This can be a simple literal (constant) value,
+a reference to a query variable that is in scope,
+a parenthesized expression,
+a function call,
+an expression accessing the ADM contents of a dataset,
+a newly constructed list of ADM instances,
+or a newly constructed ADM record.
+
+#### Literals
+
+    Literal ::= StringLiteral
+              | <INTEGER_LITERAL>
+              | <FLOAT_LITERAL>
+              | <DOUBLE_LITERAL>
+              | "null"
+              | "true"
+              | "false"
+    StringLiteral ::= <STRING_LITERAL>
+
+Literals (constants) in AQL can be strings, integers, floating point values,
+double values, boolean constants, or the constant value null.
+The null value in AQL has "unknown" or "missing" value semantics, similar to
+(though not identical to) nulls in the relational query language SQL.
+
+The following are some simple examples of AQL literals.
+Since AQL is an expression language, each example is also a complete, legal AQL query (!).
+
+##### Examples
+
+    "a string"
+    42
+
+#### Variable References
+
+    VariableRef ::= <VARIABLE>
+
+A variable in AQL can be bound to any legal ADM value.
+A variable reference refers to the value to which an in-scope variable is bound.
+(E.g., a variable binding may originate from one of the for or let clauses of a
+FLWOR expression or from an input parameter in the context of an AQL function body.)
+
+##### Examples
+
+    $tweet
+    $id
+
+#### Parenthesized Expressions
+
+    ParenthesizedExpression ::= "(" Expression ")"
+
+As in most languages, an expression may be parenthesized.
+
+Since AQL is an expression language, the following example expression is
+actually also a complete, legal AQL query whose result is the value 2.
+(As such, you can have Big Fun explaining to your boss how AsterixDB and AQL can turn
+your 1000-node shared-nothing Big Data cluster into a $5M calculator in its spare time.)
+
+##### Example
+
+    ( 1 + 1 )
+
+#### Function Calls
+
+    FunctionCallExpr ::= FunctionOrTypeName "(" ( Expression ( "," Expression )* )? ")"
+
+Functions are included in AQL, like most languages, as a way to package useful
+functionality or to componentize complicated or reusable AQL computations.
+A function call is a legal AQL query expression that represents the ADM value
+resulting from the evaluation of its body expression with the given parameter
+bindings; the parameter value bindings can themselves be any AQL expressions.
+
+The following example is a (built-in) function call expression whose value is 8.
+
+##### Example
+
+    string-length("a string")
+
+#### Dataset Access
+
+    DatasetAccessExpression ::= "dataset" ( ( Identifier ( "." Identifier )? )
+                              | ( "(" Expression ")" ) )
+    Identifier              ::= <IDENTIFIER> | StringLiteral
+
+Querying Big Data is the main point of AsterixDB and AQL.
+Data in AsterixDB reside in datasets (collections of ADM records),
+each of which in turn resides in some namespace known as a dataverse (data universe).
+Data access in a query expression is accomplished via a DatasetAccessExpression.
+Dataset access expressions are most commonly used in FLWOR expressions, where variables
+are bound to their contents.
+
+
+The following are three examples of legal dataset access expressions.
+The first one accesses a dataset called Customers in the dataverse called SalesDV.
+The second one accesses the Customers dataverse in whatever the current dataverse is.
+The third one does the same thing as the second but uses a slightly older AQL syntax.
+
+##### Examples
+
+    dataset SalesDV.Customers
+    dataset Customers
+    dataset("Customers")
+
+#### Constructors
+
+    ListConstructor          ::= ( OrderedListConstructor | UnorderedListConstructor )
+    OrderedListConstructor   ::= "[" ( Expression ( "," Expression )* )? "]"
+    UnorderedListConstructor ::= "{{" ( Expression ( "," Expression )* )? "}}"
+    RecordConstructor        ::= "{" ( FieldBinding ( "," FieldBinding )* )? "}"
+    FieldBinding             ::= Expression ":" Expression
+
+A major feature of AQL is its ability to construct new ADM data instances.
+This is accomplished using its constructors for each of the major ADM complex object structures,
+namely lists (ordered or unordered) and records.
+Ordered lists are like JSON arrays, while unordered lists have bag (multiset) semantics.
+Records are built from attributes that are field-name/field-value pairs, again like JSON.
+(See the AsterixDB Data Model document for more details on each.)
+
+The following examples illustrate how to construct a new ordered list with 3 items,
+a new unordered list with 4 items, and a new record with 2 fields, respectively.
+List elements can be homogeneous (as in the first example), which is the common case,
+or they may be heterogeneous (as in the second example).
+The data values and field name values used to construct lists and records in constructors are all simply AQL expressions.
+Thus the list elements, field names, and field values used in constructors can be simple literals (as in these three examples)
+or they can come from query variable references or even arbitrarily complex AQL expressions.
+
+##### Examples
+
+    [ "a", "b", "c" ]
+
+    {{ 42, "forty-two", "AsterixDB!", 3.14f }}
+
+    {
+      "project name": "AsterixDB"
+      "project members": {{ "vinayakb", "dtabass", "chenli" }}
+    }
+
+### Path Expressions
+
+    ValueExpr ::= PrimaryExpr ( Field | Index )*
+    Field     ::= "." Identifier
+    Index     ::= "[" ( Expression | "?" ) "]"
+
+Components of complex types in ADM are accessed via path expressions.
+Path access can be applied to the result of an AQL expression that yields an instance of such a type, e.g., a record or list instance.
+For records, path access is based on field names.
+For ordered lists, path access is based on (zero-based) array-style indexing.
+AQL also supports an "I'm feeling lucky" style index accessor, [?], for selecting an arbitrary element from an ordered list.
+Attempts to access non-existent fields or list elements produce a null (i.e., missing information) result as opposed to signaling a runtime error.
+
+The following examples illustrate field access for a record, index-based element access for an ordered list, and also a composition thereof.
+
+##### Examples
+
+    ({"list": [ "a", "b", "c"]}).list
+
+    (["a", "b", "c"])[2]
+
+    ({ "list": [ "a", "b", "c"]}).list[2]
+
+### Logical Expressions
+
+    OperatorExpr ::= AndExpr ( "or" AndExpr )*
+    AndExpr      ::= RelExpr ( "and" RelExpr )*
+
+As in most languages, boolean expressions can be built up from smaller expressions by combining them with the logical connectives and/or.
+Legal boolean values in AQL are true, false, and null.
+(Nulls in AQL are treated much like SQL treats its unknown truth value in boolean expressions.)
+
+The following is an example of a conjuctive range predicate in AQL.
+It will yield true if $a is bound to 4, null if $a is bound to null, and false otherwise.
+
+##### Example
+
+    $a > 3 and $a < 5
+
+### Comparison Expressions
+
+    RelExpr ::= AddExpr ( ( "<" | ">" | "<=" | ">=" | "=" | "!=" | "~=" ) AddExpr )?
+
+AQL has the usual list of suspects, plus one, for comparing pairs of atomic values.
+The "plus one" is the last operator listed above, which is the "roughly equal" operator provided for similarity queries.
+(See the separate document on [AsterixDB Similarity Queries](similarity.html) for more details on similarity matching.)
+
+An example comparison expression (which yields the boolean value true) is shown below.
+
+##### Example
+
+    5 > 3
+
+### Arithmetic Expressions
+
+    AddExpr  ::= MultExpr ( ( "+" | "-" ) MultExpr )*
+    MultExpr ::= UnaryExpr ( ( "*" | "/" | "%" | <CARET> | "idiv" ) UnaryExpr )*
+    UnaryExpr ::= ( ( "+" | "-" ) )? ValueExpr
+
+AQL also supports the usual cast of characters for arithmetic expressions.
+The example below evaluates to 25.
+
+##### Example
+
+    3 ^ 2 + 4 ^ 2
+
+###  FLWOR Expression
+
+    FLWOR         ::= ( ForClause | LetClause ) ( Clause )* "return" Expression
+    Clause         ::= ForClause | LetClause | WhereClause | OrderbyClause
+                     | GroupClause | LimitClause | DistinctClause
+    ForClause      ::= "for" Variable ( "at" Variable )? "in" ( Expression )
+    LetClause      ::= "let" Variable ":=" Expression
+    WhereClause    ::= "where" Expression
+    OrderbyClause  ::= "order" "by" Expression ( ( "asc" ) | ( "desc" ) )? 
+                       ( "," Expression ( ( "asc" ) | ( "desc" ) )? )*
+    GroupClause    ::= "group" "by" ( Variable ":=" )? Expression ( "," ( Variable ":=" )? Expression )*
+                       "with" VariableRef ( "," VariableRef )*
+    LimitClause    ::= "limit" Expression ( "offset" Expression )?
+    DistinctClause ::= "distinct" "by" Expression ( "," Expression )*
+    Variable       ::= <VARIABLE>
+
+The heart of AQL is the FLWOR (for-let-where-orderby-return) expression.
+The roots of this expression were borrowed from the expression of the same name in XQuery.
+A FLWOR expression starts with one or more clauses that establish variable bindings.
+A `for` clause binds a variable incrementally to each element of its associated expression;
+it includes an optional positional variable for counting/numbering the bindings.
+By default no ordering is implied or assumed by a `for` clause.
+A `let` clause binds a variable to the collection of elements computed by its associated expression.
+
+Following the initial `for` or `let` clause(s), a FLWOR expression may contain an arbitrary sequence of other clauses.
+The `where` clause in a FLWOR expression filters the preceding bindings via a boolean expression, much like a `where` clause does in a SQL query.
+The `order by` clause in a FLWOR expression induces an ordering on the data.
+The `group by` clause, discussed further below, forms groups based on its group by expressions,
+optionally naming the expressions' values (which together form the grouping key for the expression).
+The `with` subclause of a `group by` clause specifies the variable(s) whose values should be grouped based
+on the grouping key(s); following the grouping clause, only the grouping key(s) and the variables named
+in the with subclause remain in scope, and the named grouping variables now contain lists formed from their input values.
+The `limit` clause caps the number of values returned, optionally starting its result count from a specified offset.
+(Web applications can use this feature for doing pagination.)
+The `distinct` clause is similar to the `group-by` clause, but it forms no groups; it serves only to eliminate duplicate values.
+As indicated by the grammar, the clauses in an AQL query can appear in any order.
+To interpret a query, one can think of data as flowing down through the query from the first clause to the `return` clause.
+
+The following example shows a FLWOR expression that selects and returns one user from the dataset FacebookUsers.
+
+##### Example
+
+    for $user in dataset FacebookUsers
+    where $user.id = 8
+    return $user
+
+The next example shows a FLWOR expression that joins two datasets, FacebookUsers and FacebookMessages,
+returning user/message pairs.
+The results contain one record per pair, with result records containing the user's name and an entire message.
+
+##### Example
+
+    for $user in dataset FacebookUsers
+    for $message in dataset FacebookMessages
+    where $message.author-id = $user.id
+    return
+      {
+        "uname": $user.name,
+        "message": $message.message
+      };
+
+In the next example, a `let` clause is used to bind a variable to all of a user's FacebookMessages.
+The query returns one record per user, with result records containing the user's name and the set of all messages by that user.
+
+##### Example
+
+    for $user in dataset FacebookUsers
+    let $messages := 
+      for $message in dataset FacebookMessages
+      where $message.author-id = $user.id
+      return $message.message
+    return
+      {
+        "uname": $user.name,
+        "messages": $messages
+      };
+
+The following example returns all TwitterUsers ordered by their followers count (most followers first) and language.
+When ordering `null` is treated as being smaller than any other value if `null`s are encountered in the ordering key(s).
+
+##### Example
+
+      for $user in dataset TwitterUsers
+      order by $user.followers_count desc, $user.lang asc
+      return $user
+
+The next example illustrates the use of the `group by` clause in AQL.
+After the `group by` clause in the query, only variables that are either in the `group by` list or in the `with` list are in scope.
+The variables in the clause's `with` list will each contain a collection of items following the `group by` clause;
+the collected items are the values that the source variable was bound to in the tuples that formed the group.
+For grouping `null` is handled as a single value.
+
+##### Example
+
+      for $x in dataset FacebookMessages
+      let $messages := $x.message
+      group by $loc := $x.sender-location with $messages
+      return
+        {
+          "location" : $loc,
+          "message" : $messages
+        }
+
+The use of the `limit` clause is illustrated in the next example.
+
+##### Example
+
+      for $user in dataset TwitterUsers
+      order by $user.followers_count desc
+      limit 2
+      return $user
+
+The final example shows how AQL's `distinct by` clause works.
+Each variable in scope before the distinct clause is also in scope after the `distinct by` clause.
+This clause works similarly to `group by`, but for each variable that contains more than
+one value after the `distinct by` clause, one value is picked nondeterministically.
+(If the variable is in the `distinct by` list, then its value will be deterministic.)
+Nulls are treated as a single value when they occur in a grouping field.
+
+##### Example
+
+      for $x in dataset FacebookMessages
+      distinct by $x.sender-location
+      return
+        {
+          "location" : $x.sender-location,
+          "message" : $x.message
+        }
+
+### Conditional Expression
+
+    IfThenElse ::= "if" "(" Expression ")" "then" Expression "else" Expression
+
+A conditional expression is useful for choosing between two alternative values based on a
+boolean condition.  If its first (`if`) expression is true, its second (`then`) expression's
+value is returned, and otherwise its third (`else`) expression is returned.
+
+The following example illustrates the form of a conditional expression.
+##### Example
+
+    if (2 < 3) then "yes" else "no"
+
+### Quantified Expressions
+
+    QuantifiedExpression ::= ( ( "some" ) | ( "every" ) ) Variable "in" Expression
+                             ( "," Variable "in" Expression )* "satisfies" Expression
+      
+Quantified expressions are used for expressing existential or universal predicates involving the elements of a collection.
+
+The following pair of examples illustrate the use of a quantified expression to test that every (or some) element in the set [1, 2, 3] of integers is less than three. 
+The first example yields `false` and second example yields `true`.
+
+It is useful to note that if the set were instead the empty set, the first expression would yield `true`
+("every" value in an empty set satisfies the condition) while the second expression would yield `false`
+(since there isn't "some" value, as there are no values in the set, that satisfies the condition).
+
+##### Examples
+
+    every $x in [ 1, 2, 3 ] satisfies $x < 3
+    some $x in [ 1, 2, 3 ] satisfies $x < 3
+
+## 3. Statements
+
+    Statement ::= ( SingleStatement ( ";" )? )* <EOF>
+    SingleStatement ::= DataverseDeclaration
+                      | FunctionDeclaration
+                      | CreateStatement
+                      | DropStatement
+                      | LoadStatement
+                      | SetStatement
+                      | InsertStatement
+                      | DeleteStatement
+                      | Query
+
+In addition to expresssions for queries, AQL supports a variety of statements for data
+definition and manipulation purposes as well as controlling the context to be used in
+evaluating AQL expressions. 
+This section details the statement side of the AQL language.
+
+### Declarations
+ 
+    DataverseDeclaration ::= "use" "dataverse" Identifier
+
+The world of data in an AsterixDB cluster is organized into data namespaces called dataverses.
+To set the default dataverse for a series of statements, the use dataverse statement is provided.
+
+As an example, the following statement sets the default dataverse to be TinySocial.
+
+##### Example
+
+    use dataverse TinySocial;
+    
+The set statement in AQL is used to control aspects of the expression evalation context for queries.
+    
+    SetStatement ::= "set" Identifier StringLiteral
+
+As an example, the following set statements request that Jaccard similarity with a similarity threshold 0.6
+be used for set similarity matching when the ~= operator is used in a query expression.
+
+##### Example
+
+    set simfunction "jaccard";
+    set simthreshold "0.6f"; 
+
+When writing a complex AQL query, it can sometimes be helpful to define one or more
+auxilliary functions that each address a sub-piece of the overall query.
+The declare function statement supports the creation of such helper functions.
+
+    FunctionDeclaration  ::= "declare" "function" Identifier ParameterList "{" Expression "}"
+    ParameterList        ::= "(" ( <VARIABLE> ( "," <VARIABLE> )* )? ")"
+
+The following is a very simple example of a temporary AQL function definition.
+
+##### Example
+    
+    declare function add($a, $b) {
+      $a + $b
+    };
+
+### Lifecycle Management Statements
+
+    CreateStatement ::= "create" ( DataverseSpecification
+                                 | TypeSpecification
+                                 | DatasetSpecification
+                                 | IndexSpecification
+                                 | FunctionSpecification )
+
+    QualifiedName       ::= Identifier ( "." Identifier )?
+    DoubleQualifiedName ::= Identifier "." Identifier ( "." Identifier )?
+
+The create statement in AQL is used for creating persistent artifacts in the context of dataverses.
+It can be used to create new dataverses, datatypes, datasets, indexes, and user-defined AQL functions.
+
+#### Dataverses
+
+    DataverseSpecification ::= "dataverse" Identifier IfNotExists ( "with format" StringLiteral )?
+
+The create dataverse statement is used to create new dataverses.
+To ease the authoring of reusable AQL scripts, its optional IfNotExists clause allows creation
+to be requested either unconditionally or only if the the dataverse does not already exist.
+If this clause is absent, an error will be returned if the specified dataverse already exists.
+The `with format` clause is a placeholder for future functionality that can safely be ignored.
+
+The following example creates a dataverse named TinySocial.
+
+##### Example
+
+    create dataverse TinySocial;
+
+#### Types
+
+    TypeSpecification    ::= "type" FunctionOrTypeName IfNotExists "as" TypeExpr
+    FunctionOrTypeName   ::= QualifiedName
+    IfNotExists          ::= ( "if not exists" )?
+    TypeExpr             ::= RecordTypeDef | TypeReference | OrderedListTypeDef | UnorderedListTypeDef
+    RecordTypeDef        ::= ( "closed" | "open" )? "{" ( RecordField ( "," RecordField )* )? "}"
+    RecordField          ::= Identifier ":" ( TypeExpr ) ( "?" )?
+    TypeReference        ::= Identifier
+    OrderedListTypeDef   ::= "[" ( TypeExpr ) "]"
+    UnorderedListTypeDef ::= "{{" ( TypeExpr ) "}}"
+
+The create type statement is used to create a new named ADM datatype.
+This type can then be used to create datasets or utilized when defining one or more other ADM datatypes.
+Much more information about the Asterix Data Model (ADM) is available in the [data model reference guide](datamodel.html) to ADM.
+A new type can be a record type, a renaming of another type, an ordered list type, or an unordered list type.
+A record type can be defined as being either open or closed.
+Instances of a closed record type are not permitted to contain fields other than those specified in the create type statement.
+Instances of an open record type may carry additional fields, and open is the default for a new type (if neither option is specified).
+
+The following example creates a new ADM record type called FacebookUser type.
+Since it is closed, its instances will contain only what is specified in the type definition.
+The first four fields are traditional typed name/value pairs.
+The friend-ids field is an unordered list of 32-bit integers.
+The employment field is an ordered list of instances of another named record type, EmploymentType.
+
+##### Example
+
+    create type FacebookUserType as closed {
+      id:         int32,
+      alias:      string,
+      name:       string,
+      user-since: datetime,
+      friend-ids: {{ int32 }},
+      employment: [ EmploymentType ]
+    }
+
+#### Datasets
+
+    DatasetSpecification ::= "internal"? "dataset" QualifiedName "(" Identifier ")" IfNotExists
+                             PrimaryKey ( "on" Identifier )? ( "hints" Properties )? 
+                           | "external" "dataset" QualifiedName "(" Identifier ")" IfNotExists 
+                             "using" AdapterName Configuration ( "hints" Properties )?
+    AdapterName          ::= Identifier
+    Configuration        ::= "(" ( KeyValuePair ( "," KeyValuePair )* )? ")"
+    KeyValuePair         ::= "(" StringLiteral "=" StringLiteral ")"
+    Properties           ::= ( "(" Property ( "," Property )* ")" )?
+    Property             ::= Identifier "=" ( StringLiteral | <INTEGER_LITERAL> )
+    FunctionSignature    ::= FunctionOrTypeName "@" <INTEGER_LITERAL>
+    PrimaryKey           ::= "primary" "key" Identifier ( "," Identifier )*
+
+The create dataset statement is used to create a new dataset.
+Datasets are named, unordered collections of ADM record instances; they
+are where data lives persistently and are the targets for queries in AsterixDB.
+Datasets are typed, and AsterixDB will ensure that their contents conform to their type definitions.
+An Internal dataset (the default) is a dataset that is stored in and managed by AsterixDB.
+It must have a specified unique primary key that can be used to partition data across nodes of an AsterixDB cluster.
+The primary key is also used in secondary indexes to uniquely identify the indexed primary data records.
+An External dataset is stored outside of AsterixDB, e.g., in HDFS or in the local filesystem(s) of the cluster's nodes.
+External dataset support allows AQL queries to treat external data as though it were stored in AsterixDB,
+making it possible to query "legacy" file data (e.g., Hive data) without having to physically import it into AsterixDB.
+For an external dataset, an appropriate adaptor must be selected to handle the nature of the desired external data.
+(See the [guide to external data](externaldata.html) for more information on the available adaptors.)
+
+The following example creates an internal dataset for storing FacefookUserType records.
+It specifies that their id field is their primary key.
+
+##### Example
+    create internal dataset FacebookUsers(FacebookUserType) primary key id;
+
+The next example creates an external dataset for storing LineitemType records.
+The choice of the `localfs` adaptor means that its data will reside in the local filesystem of the cluster nodes.
+The create statement provides several parameters used by the localfs adaptor;
+e.g., the file format is delimited text with vertical bar being the field delimiter.
+
+##### Example
+
+    create external dataset Lineitem(LineitemType) using localfs (
+      ("path"="127.0.0.1://SOURCE_PATH"),
+      ("format"="delimited-text"),
+      ("delimiter"="|"));
+      
+#### Indices
+
+    IndexSpecification ::= "index" Identifier IfNotExists "on" QualifiedName 
+                           "(" ( Identifier ) ( "," Identifier )* ")" ( "type" IndexType )?
+    IndexType          ::= "btree"
+                         | "rtree"
+                         | "keyword"
+                         | "ngram" "(" <INTEGER_LITERAL> ")"
+
+The create index statement creates a secondary index on one or more fields of a specified dataset.
+Supported index types include `btree` for totally ordered datatypes,
+`rtree` for spatial data, and `keyword` and `ngram` for textual (string) data.
+AsterixDB currently requires indexed fields to be part of the named type associated with a dataset.
+(Future plans include support for indexing of open fields as well.)
+
+The following example creates a btree index called fbAuthorIdx on the author-id field of the FacebookMessages dataset.
+This index can be useful for accelerating exact-match queries, range search queries, and joins involving the author-id field.
+
+##### Example
+
+    create index fbAuthorIdx on FacebookMessages(author-id) type btree;
+
+The following example creates an rtree index called fbSenderLocIdx on the sender-location field of the FacebookMessages dataset.
+This index can be useful for accelerating queries that use the
+[`spatial-intersect` function](functions.html#spatial-intersect) in a predicate involving the 
+sender-location field.
+
+##### Example
+
+    create index fbSenderLocIndex on FacebookMessages(sender-location) type rtree;
+
+The following example creates a 3-gram index called fbUserIdx on the name field of the FacebookUsers dataset.
+This index can be used to accelerate some similarity or substring maching queries on the name field.
+For details refer to the [document on similarity queries](similarity.html#NGram_Index).
+
+##### Example
+
+    create index fbUserIdx on FacebookUsers(name) type ngram(3);
+
+The following example creates a keyword index called fbMessageIdx on the message field of the FacebookMessages dataset.
+This keyword index can be used to optimize queries with token-based similarity predicates on the message field.
+For details refer to the [document on similarity queries](similarity.html#Keyword_Index).
+
+##### Example
+
+    create index fbMessageIdx on FacebookMessages(message) type keyword;
+
+#### Functions
+
+The create function statement creates a named function that can then be used and reused in AQL queries.
+The body of a function can be any AQL expression involving the function's parameters.
+
+    FunctionSpecification ::= "function" FunctionOrTypeName IfNotExists ParameterList "{" Expression "}"
+
+The following is a very simple example of a create function statement.
+It differs from the declare function example shown previously in that it results in a function that is
+persistently registered by name in the specified dataverse.
+    
+##### Example
+    
+    create function add($a, $b) {
+      $a + $b
+    };
+    
+#### Removal
+
+    DropStatement       ::= "drop" ( "dataverse" Identifier IfExists
+                                   | "type" FunctionOrTypeName IfExists
+                                   | "dataset" QualifiedName IfExists
+                                   | "index" DoubleQualifiedName IfExists
+                                   | "function" FunctionSignature IfExists )
+    IfExists            ::= ( "if" "exists" )?
+
+The drop statement in AQL is the inverse of the create statement.
+It can be used to drop dataverses, datatypes, datasets, indexes, and functions.
+
+The following examples illustrate uses of the drop statement.
+ 
+##### Example
+
+    drop dataset FacebookUsers if exists;
+
+    drop index fbSenderLocIndex;
+
+    drop type FacebookUserType;
+    
+    drop dataverse TinySocial;
+
+    drop function add;
+
+### Import/Export Statements
+
+    LoadStatement  ::= "load" "dataset" QualifiedName "using" AdapterName Configuration ( "pre-sorted" )?
+    
+The load statement is used to initially populate a dataset via bulk loading of data from an external file.
+An appropriate adaptor must be selected to handle the nature of the desired external data.
+(See the [guide to external data](externaldata.html) for more information on the available adaptors.)
+
+The following example shows how to bulk load the FacebookUsers dataset from an external file containing
+data that has been prepared in ADM format.
+
+##### Example
+
+    load dataset FacebookUsers using localfs
+    (("path"="localhost:///Users/zuck/AsterixDB/load/fbu.adm"),("format"="adm"));
+
+### Modification Statements
+
+#### Insert
+
+    InsertStatement ::= "insert" "into" "dataset" QualifiedName Query
+
+The AQL insert statement is used to insert data into a dataset.
+The data to be inserted comes from an AQL query expression.
+The expression can be as simple as a constant expression, or in general it can be any legal AQL query.
+Inserts in AsterixDB are processed transactionally, with the scope of each insert transaction
+being the insertion of a single object plus its affiliated secondary index entries (if any).
+If the query part of an insert returns a single object, then the insert statement itself will
+be a single, atomic transaction.
+If the query part returns multiple objects, then each object inserted will be handled independently
+as a tranaction.
+
+The following example illustrates a query-based insertion.
+    
+##### Example
+
+    insert into dataset UsersCopy (for $user in dataset FacebookUsers return $user
+
+#### Delete
+
+    DeleteStatement ::= "delete" Variable "from" "dataset" QualifiedName ( "where" Expression )?
+
+The AQL delete statement is used to delete data from a target dataset.
+The data to be deleted is identified by a boolean expression involving the variable bound to the
+target dataset in the delete statement.
+Deletes in AsterixDB are processed transactionally, with the scope of each delete transaction
+being the deletion of a single object plus its affiliated secondary index entries (if any).
+If the boolean expression for a delete identifies a single object, then the delete statement itself
+will be a single, atomic transaction.
+If the expression identifies multiple objects, then each object deleted will be handled independently
+as a transaction.
+
+The following example illustrates a single-object deletion.
+
+##### Example
+    
+    delete $user from dataset FacebookUsers where $user.id = 8;
+
+We close this guide to AQL with one final example of a query expression.
+    
+##### Example
+    
+    for $praise in {{ "great", "brilliant", "awesome" }}
+    return
+       string-concat(["AsterixDB is ", $praise]
+
diff --git a/asterix-doc/src/site/markdown/AdmAql101.md b/asterix-doc/src/site/markdown/aql/primer.md
similarity index 98%
rename from asterix-doc/src/site/markdown/AdmAql101.md
rename to asterix-doc/src/site/markdown/aql/primer.md
index ed4736e..fc8ea9a 100644
--- a/asterix-doc/src/site/markdown/AdmAql101.md
+++ b/asterix-doc/src/site/markdown/aql/primer.md
@@ -12,7 +12,7 @@
 Most importantly, it assumes you already have a running instance of AsterixDB and that you know how to query
 it using AsterixDB's basic web interface.
 For more information on these topics, you should go through the steps in 
-[Installing Asterix Using Managix](InstallingAsterixUsingManagix.html)
+[Installing Asterix Using Managix](../install.html)
 before reading this document and make sure that you have a running AsterixDB instance ready to go.
 To get your feet wet, you should probably start with a simple local installation of AsterixDB on your favorite
 machine, accepting all of the default settings that Managix offers.
@@ -211,8 +211,8 @@
 This index will be a B+ tree index; its type is unspecified and _btree_ is the default type.
 The other three illustrate how you can explicitly specify the desired type of index.
 In addition to btree, _rtree_ and inverted _keyword_ indexes are supported by AsterixDB.
-Indexes can also have composite keys, and there are more advanced flavors of text indexing
-available as well (_fuzzy keyword_ and _ngram(k)_, where _k_ is the desired gram length).
+Indexes can also have composite keys, and more advanced text indexing is available as well
+(ngram(k), where k is the desired gram length).
 
 ### Querying the Metadata Dataverse ###
 
@@ -350,7 +350,7 @@
 in the not-too-distant future we will also provide a complete reference manual for the language.
 In the meantime, this will get you started down the path of using AsterixDB.
 A more complete list of the supported AsterixDB primitive types and built-in functions can be
-found at [AsterixDataTypesAndFunctions](AsterixDataTypesAndFunctions.html).
+found at [Asterix Data Model (ADM)](datamodel.html) and [Asterix Functions](functions.html).
 
 AQL is an expression language.
 Even the expression 1+1 is a valid AQL query that evaluates to 2.
diff --git a/asterix-doc/src/site/markdown/aql/similarity.md b/asterix-doc/src/site/markdown/aql/similarity.md
new file mode 100644
index 0000000..244103c
--- /dev/null
+++ b/asterix-doc/src/site/markdown/aql/similarity.md
@@ -0,0 +1,174 @@
+
+# AsterixDB  Support of Similarity Queries #
+
+## Motivation ##
+
+Similarity queries are widely used in applications where users need to
+find records that satisfy a similarity predicate, while exact matching
+is not sufficient. These queries are especially important for social
+and Web applications, where errors, abbreviations, and inconsistencies
+are common.  As an example, we may want to find all the movies
+starring Schwarzenegger, while we don't know the exact spelling of his
+last name (despite his popularity in both the movie industry and
+politics :-)). As another example, we want to find all the Facebook
+users who have similar friends. To meet this type of needs, AsterixDB
+supports similarity queries using efficient indexes and algorithms.
+
+## Data Types and Similarity Functions ##
+
+AsterixDB supports [edit distance](http://en.wikipedia.org/wiki/Levenshtein_distance) (on strings) and
+[Jaccard](http://en.wikipedia.org/wiki/Jaccard_index) (on sets).  For
+instance, in our
+[TinySocial](primer.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB)
+example, the `friend-ids` of a Facebook user forms a set
+of friends, and we can define a similarity between the sets of
+friends of two users. We can also convert a string to a set of grams of a length "n"
+(called "n-grams") and define the Jaccard similarity between the two
+gram sets of the two strings. Formally, the "n-grams" of a string are
+its substrings of length "n". For instance, the 3-grams of the string
+`schwarzenegger` are `sch`, `chw`, `hwa`, ..., `ger`.
+
+AsterixDB provides
+[tokenization functions](functions.html#Tokenizing_Functions)
+to convert strings to sets, and the
+[similarity functions](functions.html#Similarity_Functions).
+
+## Similarity Selection Queries ##
+
+The following [query](functions.html#edit-distance)
+asks for all the Facebook users whose name is similar to
+`Suzanna Tilson`, i.e., their edit distance is at most 2.
+
+        use dataverse TinySocial;
+        
+        for $user in dataset('FacebookUsers')
+        let $ed := edit-distance($user.name, "Suzanna Tilson")
+        where $ed <= 2
+        return $user
+
+
+The following [query](functions.html#similarity-jaccard)
+asks for all the Facebook users whose set of friend ids is
+similar to `[1,5,9]`, i.e., their Jaccard similarity is at least 0.6.
+
+        use dataverse TinySocial;
+        
+        for $user in dataset('FacebookUsers')
+        let $sim := similarity-jaccard($user.friend-ids, [1,5,9])
+        where $sim >= 0.6f
+        return $user
+
+
+AsterixDB allows a user to use a similarity operator `~=` to express a
+condition by defining the similarity function and threshold
+using "set" statements earlier. For instance, the above query can be
+equivalently written as:
+
+        use dataverse TinySocial;
+        
+        set simfunction "jaccard";
+        set simthreshold "0.6f";
+        
+        for $user in dataset('FacebookUsers')
+        where $user.friend-ids ~= [1,5,9]
+        return $user
+
+
+In this query, we first declare Jaccard as the similarity function
+using `simfunction` and then specify the threshold `0.6f` using
+`simthreshold`.
+
+## Similarity Join Queries ##
+
+AsterixDB supports fuzzy joins between two sets. The following
+[query](primer.html#Query_5_-_Fuzzy_Join)
+finds, for each Facebook user, all Twitter users with names
+similar to their name based on the edit distance.
+
+        use dataverse TinySocial;
+        
+        set simfunction "edit-distance";
+        set simthreshold "3";
+	
+        for $fbu in dataset FacebookUsers
+        return {
+            "id": $fbu.id,
+            "name": $fbu.name,
+            "similar-users": for $t in dataset TweetMessages
+                                let $tu := $t.user
+                                where $tu.name ~= $fbu.name
+                                return {
+                                "twitter-screenname": $tu.screen-name,
+                                "twitter-name": $tu.name
+                                }
+        };
+
+## Using Indexes to Support Similarity Queries ##
+
+AsterixDB uses two types of indexes to support similarity queries, namely
+"ngram index" and "keyword index".
+
+### NGram Index ###
+
+An "ngram index" is constructed on a set of strings.  We generate n-grams for each string, and build an inverted
+list for each n-gram that includes the ids of the strings with this
+gram.  A similarity query can be answered efficiently by accessing the
+inverted lists of the grams in the query and counting the number of
+occurrences of the string ids on these inverted lists.  The similar
+idea can be used to answer queries with Jaccard similarity.  A
+detailed description of these techniques is available at this
+[paper](http://www.ics.uci.edu/~chenli/pub/icde2009-memreducer.pdf).
+
+For instance, the following DDL statements create an ngram index on the
+`FacebookUsers.name` attribute using an inverted index of 3-grams.
+
+        use dataverse TinySocial;
+        
+        create index fbUserIdx on FacebookUsers(name) type ngram(3);
+
+The number "3" in "ngram(3)" is the length "n" in the grams. This
+index can be used to optimize similarity queries on this attribute
+using 
+[edit-distance](functions.html#edit-distance), 
+[edit-distance-check](functions.html#edit-distance-check), 
+[jaccard](functions.html#similarity-jaccard),
+or [jaccard-check](functions.html#similarity-jaccard-check) 
+queries on this attribute where the
+similarity is defined on sets of 3-grams.  This index can also be used
+to optimize queries with the "[contains()]((functions.html#contains))" predicate (i.e., substring
+matching) since it can be also be solved by counting on the inverted
+lists of the grams in the query string.
+
+### Keyword Index ###
+
+A "keyword index" is constructed on a set of strings or sets (e.g., OrderedList, UnorderedList). Instead of 
+generating grams as in an ngram index, we generate tokens (e.g., words) and for each token, construct an inverted list that includes the ids of the
+records with this token.  The following two examples show how to create keyword index on two different types:
+
+
+#### Keyword Index on String Type ####
+
+        use dataverse TinySocial;
+
+        create index fbMessageIdx on FacebookMessages(message) type keyword;
+
+        for $o in dataset('FacebookMessages')
+        let $jacc := similarity-jaccard-check(word-tokens($o.message), word-tokens("love like verizon"), 0.2f)
+        where $jacc[0]
+        return $o
+        
+#### Keyword Index on UnorderedList Type ####
+        
+        use dataverse TinySocial;
+
+        create index fbUserIdx_fids on FacebookUsers(friend-ids) type keyword;
+
+        for $c in dataset('FacebookUsers')
+        let $jacc := similarity-jaccard-check($c.friend-ids, {{3,10}}, 0.5f)
+        where $jacc[0]
+        return $c
+        
+As shown above, keyword index can be used to optimize queries with token-based similarity predicates, including
+[similarity-jaccard](functions.html#similarity-jaccard) and
+[similarity-jaccard-check](functions.html#similarity-jaccard-check).
+
diff --git a/asterix-doc/src/site/markdown/index.md b/asterix-doc/src/site/markdown/index.md
index cba2fdc..4ee2a5f 100644
--- a/asterix-doc/src/site/markdown/index.md
+++ b/asterix-doc/src/site/markdown/index.md
@@ -1 +1,54 @@
-# AsterixDB
+# AsterixDB: A Big Data Management System #
+
+## What Is AsterixDB? ##
+
+Welcome to the new home of the AsterixDB Big Data Management System (BDMS).
+The AsterixDB BDMS is the result of about 3.5 years of R&D involving researchers at UC Irvine, UC Riverside, and UC San Diego.
+The AsterixDB code base now consists of roughly 250K lines of Java code that has been co-developed at UC Irvine and UC Riverside.
+
+Initiated in 2009, the NSF-sponsored ASTERIX project has been developing new technologies for ingesting, storing, managing, indexing, querying, and analyzing vast quantities of semi-structured information.
+The project has been combining ideas from three distinct areas---semi-structured data, parallel databases, and data-intensive computing (a.k.a. today's Big Data platforms)---in order to create a next-generation, open-source software platform that scales by running on large, shared-nothing commodity computing clusters.
+The ASTERIX effort has been targeting a wide range of semi-structured information, ranging from "data" use cases---where information is well-typed and highly regular---to "content" use cases---where data tends to be irregular, much of each datum may be textual, and the ultimate schema for the various data types involved may be hard to anticipate up front.
+The ASTERIX project has been addressing technical issues including highly scalable data storage and indexing,  semi-structured query processing on very large clusters, and  merging time-tested parallel database techniques with modern data-intensive computing techniques  to support performant yet declarative solutions to the problem of storing and analyzing semi-structured information effectively.
+The first fruits of this labor have been captured in the AsterixDB system that is now being released in preliminary or "Beta" release form.
+We are hoping that the arrival of AsterixDB will mark the beginning of the "BDMS era", and we hope that both the Big Data community and the database community will find the AsterixDB system to be interesting and useful for a much broader class of problems than can be addressed with any one of today's current Big Data platforms and related technologies (e.g., Hadoop, Pig, Hive, HBase, MongoDB, and so on).  One of our project mottos has been "one size fits a bunch"---at least that has been our aim.  For more information about the research effort that led to the birth of AsterixDB, please refer to our NSF project web site: [http://asterix.ics.uci.edu/](http://asterix.ics.uci.edu/).
+
+In a nutshell, AsterixDB is a full-function BDMS with a rich feature set that distinguishes it from pretty much any other Big Data platform that's out and available today.  We believe that its feature set makes it well-suited to modern needs such as web data warehousing and social data storage and analysis.  AsterixDB has:
+
+ * A semistructured NoSQL style data model (ADM) resulting from extending JSON with object database ideas
+ * An expressive and declarative query language (AQL) that supports a broad range of queries and analysis over semistructured data
+ * A parallel runtime query execution engine, Hyracks, that has been scale-tested on up to 1000+ cores and 500+ disks
+ * Partitioned LSM-based data storage and indexing to support efficient ingestion and management of semistructured data
+ * Support for query access to externally stored data (e.g., data in HDFS) as well as to data stored natively by AsterixDB
+ * A rich set of primitive data types, including spatial and temporal data in addition to integer, floating point, and textual data
+ * Secondary indexing options that include B+ trees, R trees, and inverted keyword (exact and fuzzy) index types
+ * Support for fuzzy and spatial queries as well as for more traditional parametric queries
+ * Basic transactional (concurrency and recovery) capabilities akin to those of a NoSQL store
+
+## Getting and Using AsterixDB ##
+
+You are most likely here because you are interested in getting your hands on AsterixDB---so you would like to know how to get it, how to set it up, and how to use it.
+Someday our plan is to have comprehensive documentation for AsterixDB and its data model (ADM) and query language (AQL) here on this wiki.
+For the Beta release, we've got a start; for the Beta release a month or so from now, we will hopefully have much more.
+The following is a list of the wiki pages and supporting documents that we have available today:
+
+1. [Installing AsterixDB using Managix](install.html) :
+This is our installation guide, and it is where you should start.
+This document will tell you how to obtain, install, and manage instances of [AsterixDB](https://asterixdb.googlecode.com/files/asterix-installer-0.0.4-binary-assembly.zip), including both single-machine setup (for developers) as well as cluster installations (for deployment in its intended form).
+
+2. [AsterixDB 101: An ADM and AQL Primer](aql/primer.html) :
+This is a first-timers introduction to the user model of the AsterixDB BDMS, by which we mean the view of AsterixDB as seen from the perspective of an "average user" or Big Data application developer.
+The AsterixDB user model consists of its data modeling features (ADM) and its query capabilities (AQL).
+This document presents a tiny "social data warehousing" example and uses it as a backdrop for describing, by example, the key features of AsterixDB.
+By working through this document, you will learn how to define the artifacts needed to manage data in AsterixDB, how to load data into the system, how to use most of the basic features of its query language, and how to insert and delete data dynamically.
+
+3. [Asterix Data Model (ADM)](aql/datamodel.html), [Asterix Functions](aql/functions.html), and [Asterix Query Language (AQL)](aql/manual.html) :
+These are reference documents that catalog the primitive data types and built-in functions available in AQL and the reference manual for AQL itself.
+
+5. [REST API to AsterixDB](api.html) :
+Access to data in an AsterixDB instance is provided via a REST-based API.
+This is a short document that describes the REST API entry points and their URL syntax.
+
+To all who have now come this far: Thanks for your interest in AsterixDB, and for kicking its tires in its Beta form.
+In addition to getting the system and trying it out, please sign up as a member of the AsterixDB user mailing list (asterixdb-users (at) googlegroups.com) so that you can contact us easily with your questions, issues, and other feedback.
+We want AsterixDB to be a "big hit" some day, and we are anxious to see what users do with it and to learn from that feedback what we should be working on most urgently in the next phase of the project.
diff --git a/asterix-doc/src/site/markdown/InstallingAsterixUsingManagix.md b/asterix-doc/src/site/markdown/install.md
similarity index 100%
rename from asterix-doc/src/site/markdown/InstallingAsterixUsingManagix.md
rename to asterix-doc/src/site/markdown/install.md
diff --git a/asterix-doc/src/site/resources/images/asterixlogo.png b/asterix-doc/src/site/resources/images/asterixlogo.png
new file mode 100644
index 0000000..45cd64f
--- /dev/null
+++ b/asterix-doc/src/site/resources/images/asterixlogo.png
Binary files differ
diff --git a/asterix-doc/src/site/resources/images/hyrax_ts.png b/asterix-doc/src/site/resources/images/hyrax_ts.png
new file mode 100644
index 0000000..001c788
--- /dev/null
+++ b/asterix-doc/src/site/resources/images/hyrax_ts.png
Binary files differ
diff --git a/asterix-doc/src/site/site.xml b/asterix-doc/src/site/site.xml
index 6724153..017bf10 100644
--- a/asterix-doc/src/site/site.xml
+++ b/asterix-doc/src/site/site.xml
@@ -3,22 +3,37 @@
 <project name="AsterixDB" xmlns="http://maven.apache.org/DECORATION/1.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/DECORATION/1.0.0 http://maven.apache.org/xsd/decoration-1.0.0.xsd">
-  <!--
   <bannerLeft>
-    <name>Asterix</name>
-    <src>http://asterix.ics.uci.edu/pic/img9.jpg</src>
-    <href>http://asterix.ics.uci.edu/</href>
+    <name>AsterixDB</name>
+    <src>images/asterixlogo.png</src>
+    <href>/index.html</href>
   </bannerLeft>
-  -->
+  
+  <version position="right"/>
+
+  <poweredBy>
+    <logo name="Hyracks"
+          href="https://code.google.com/p/hyracks/"
+          img="images/hyrax_ts.png"/>
+  </poweredBy>
 
   <skin>
     <groupId>org.apache.maven.skins</groupId>
     <artifactId>maven-fluido-skin</artifactId>
-    <version>1.2.1</version>
+    <version>1.3.0</version>
   </skin>
   <custom>
     <fluidoSkin>
-      <sideBarEnabled>true</sideBarEnabled>
+      <!-- <topBarEnabled>true</topBarEnabled>
+      <topBarIcon>
+        <name>AsterixDB</name>
+        <alt>AsterixDB</alt>
+        <src>images/asterixlogo.png</src>
+        <href>/index.html</href>
+      </topBarIcon>
+      <sideBarEnabled>false</sideBarEnabled> -->
+      <!-- <topBarContainerStyle>width: 90%;</topBarContainerStyle> -->
+      <sourceLineNumbersEnabled>true</sourceLineNumbersEnabled>
       <!-- <googlePlusOne /> -->
     </fluidoSkin>
   </custom>
@@ -29,18 +44,19 @@
     </links>
 
     <menu name="Documentation">
-      <item name="AsterixDB: A Big Data Management System" href="AsterixAlphaRelease.html"/>
-      <item name="Installing Asterix using Managix" href="InstallingAsterixUsingManagix.html"/>
-      <item name="AsterixDB 101: An ADM and AQL Primer" href="AdmAql101.html"/>
-      <item name="Asterix Data Model (ADM)" href="AsterixDBDataModel.html"/>
-      <item name="AsterixDB Functions" href="AsterixDBFunctions.html"/>
-      <item name="The Asterix Query Language" href="AsterixQueryLanguageReference.html"/>
-      <item name="AsterixDB Support of Similarity Queries" href="AsterixSimilarityQueries.html"/>
-      <item name="Accessing External Data in AsterixDB" href="AccessingExternalDataInAsterixDB.html"/>
-      <item name="REST API to AsterixDB" href="AsterixDBRestAPI.html"/>
-      <item name="(old AQL doc)" href="AsterixQueryLanguage.html"/>      
+      <item name="Installing AsterixDB using Managix" href="install.html"/>
+      <item name="AsterixDB 101: An ADM and AQL Primer" href="aql/primer.html"/>
+      <item name="Asterix Data Model (ADM)" href="aql/datamodel.html"/>
+      <item name="Asterix Functions" href="aql/functions.html"/>
+      <item name="Asterix Query Language (AQL)" href="aql/manual.html"/>
+      <item name="AQL Support of Similarity Queries" href="aql/similarity.html"/>
+      <item name="Accessing External Data" href="aql/externaldata.html"/>
+      <item name="REST API to AsterixDB" href="api.html"/>
     </menu>
 
     <menu ref="reports"/>
+
+    <footer>&copy; Copyright 2013 University of California, Irvine</footer>
+
   </body>
 </project>
diff --git a/asterix-events/.gitignore b/asterix-events/.gitignore
new file mode 100644
index 0000000..ea8c4bf
--- /dev/null
+++ b/asterix-events/.gitignore
@@ -0,0 +1 @@
+/target
diff --git a/asterix-events/pom.xml b/asterix-events/pom.xml
index 94042e1..108bde2 100644
--- a/asterix-events/pom.xml
+++ b/asterix-events/pom.xml
@@ -173,5 +173,10 @@
      <type>jar</type>
      <scope>compile</scope>
     </dependency>
+   <dependency>
+   	<groupId>log4j</groupId>
+   	<artifactId>log4j</artifactId>
+   	<version>1.2.9</version>
+   </dependency>
   </dependencies>
 </project>
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
index e702ef3..63a791b 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
@@ -18,7 +18,6 @@
 import java.io.File;
 import java.io.IOException;
 import java.io.StringWriter;
-import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.List;
 
diff --git a/asterix-examples/src/main/resources/tiny-social/data/fbm.adm b/asterix-examples/src/main/resources/tiny-social/data/fbm.adm
new file mode 100644
index 0000000..58d7f08
--- /dev/null
+++ b/asterix-examples/src/main/resources/tiny-social/data/fbm.adm
@@ -0,0 +1,15 @@
+{"message-id":1,"author-id":3,"in-response-to":2,"sender-location":point("47.16,77.75"),"message":" love sprint its shortcut-menu is awesome:)"}
+{"message-id":2,"author-id":1,"in-response-to":4,"sender-location":point("41.66,80.87"),"message":" dislike iphone its touch-screen is horrible"}
+{"message-id":3,"author-id":2,"in-response-to":4,"sender-location":point("48.09,81.01"),"message":" like samsung the plan is amazing"}
+{"message-id":4,"author-id":1,"in-response-to":2,"sender-location":point("37.73,97.04"),"message":" can't stand at&t the network is horrible:("}
+{"message-id":5,"author-id":6,"in-response-to":2,"sender-location":point("34.7,90.76"),"message":" love sprint the customization is mind-blowing"}
+{"message-id":6,"author-id":2,"in-response-to":1,"sender-location":point("31.5,75.56"),"message":" like t-mobile its platform is mind-blowing"}
+{"message-id":7,"author-id":5,"in-response-to":15,"sender-location":point("32.91,85.05"),"message":" dislike sprint the speed is horrible"}
+{"message-id":8,"author-id":1,"in-response-to":11,"sender-location":point("40.33,80.87"),"message":" like verizon the 3G is awesome:)"}
+{"message-id":9,"author-id":3,"in-response-to":12,"sender-location":point("34.45,96.48"),"message":" love verizon its wireless is good"}
+{"message-id":10,"author-id":1,"in-response-to":12,"sender-location":point("42.5,70.01"),"message":" can't stand motorola the touch-screen is terrible"}
+{"message-id":11,"author-id":1,"in-response-to":1,"sender-location":point("38.97,77.49"),"message":" can't stand at&t its plan is terrible"}
+{"message-id":12,"author-id":10,"in-response-to":6,"sender-location":point("42.26,77.76"),"message":" can't stand t-mobile its voicemail-service is OMG:("}
+{"message-id":13,"author-id":10,"in-response-to":4,"sender-location":point("42.77,78.92"),"message":" dislike iphone the voice-command is bad:("}
+{"message-id":14,"author-id":9,"in-response-to":12,"sender-location":point("41.33,85.28"),"message":" love at&t its 3G is good:)"}
+{"message-id":15,"author-id":7,"in-response-to":11,"sender-location":point("44.47,67.11"),"message":" like iphone the voicemail-service is awesome"}
diff --git a/asterix-examples/src/main/resources/tiny-social/data/fbu.adm b/asterix-examples/src/main/resources/tiny-social/data/fbu.adm
new file mode 100644
index 0000000..7e99ea4
--- /dev/null
+++ b/asterix-examples/src/main/resources/tiny-social/data/fbu.adm
@@ -0,0 +1,10 @@
+{"id":1,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]}
+{"id":2,"alias":"Isbel","name":"IsbelDull","user-since":datetime("2011-01-22T10:10:00"),"friend-ids":{{1,4}},"employment":[{"organization-name":"Hexviafind","start-date":date("2010-04-27")}]}
+{"id":3,"alias":"Emory","name":"EmoryUnk","user-since":datetime("2012-07-10T10:10:00"),"friend-ids":{{1,5,8,9}},"employment":[{"organization-name":"geomedia","start-date":date("2010-06-17"),"end-date":date("2010-01-26")}]}
+{"id":4,"alias":"Nicholas","name":"NicholasStroh","user-since":datetime("2010-12-27T10:10:00"),"friend-ids":{{2}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2010-06-08")}]}
+{"id":5,"alias":"Von","name":"VonKemble","user-since":datetime("2010-01-05T10:10:00"),"friend-ids":{{3,6,10}},"employment":[{"organization-name":"Kongreen","start-date":date("2010-11-27")}]}
+{"id":6,"alias":"Willis","name":"WillisWynne","user-since":datetime("2005-01-17T10:10:00"),"friend-ids":{{1,3,7}},"employment":[{"organization-name":"jaydax","start-date":date("2009-05-15")}]}
+{"id":7,"alias":"Suzanna","name":"SuzannaTillson","user-since":datetime("2012-08-07T10:10:00"),"friend-ids":{{6}},"employment":[{"organization-name":"Labzatron","start-date":date("2011-04-19")}]}
+{"id":8,"alias":"Nila","name":"NilaMilliron","user-since":datetime("2008-01-01T10:10:00"),"friend-ids":{{3}},"employment":[{"organization-name":"Plexlane","start-date":date("2010-02-28")}]}
+{"id":9,"alias":"Woodrow","name":"WoodrowNehling","user-since":datetime("2005-09-20T10:10:00"),"friend-ids":{{3,10}},"employment":[{"organization-name":"Zuncan","start-date":date("2003-04-22"),"end-date":date("2009-12-13")}]}
+{"id":10,"alias":"Bram","name":"BramHatch","user-since":datetime("2010-10-16T10:10:00"),"friend-ids":{{1,5,9}},"employment":[{"organization-name":"physcane","start-date":date("2007-06-05"),"end-date":date("2011-11-05")}]}
diff --git a/asterix-examples/src/main/resources/tiny-social/data/twm.adm b/asterix-examples/src/main/resources/tiny-social/data/twm.adm
new file mode 100644
index 0000000..fa764af
--- /dev/null
+++ b/asterix-examples/src/main/resources/tiny-social/data/twm.adm
@@ -0,0 +1,12 @@
+{"tweetid":"1","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("47.44,80.65"),"send-time":datetime("2008-04-26T10:10:00"),"referred-topics":{{"t-mobile","customization"}},"message-text":" love t-mobile its customization is good:)"}
+{"tweetid":"2","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("32.84,67.14"),"send-time":datetime("2010-05-13T10:10:00"),"referred-topics":{{"verizon","shortcut-menu"}},"message-text":" like verizon its shortcut-menu is awesome:)"}
+{"tweetid":"3","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("29.72,75.8"),"send-time":datetime("2006-11-04T10:10:00"),"referred-topics":{{"motorola","speed"}},"message-text":" like motorola the speed is good:)"}
+{"tweetid":"4","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("39.28,70.48"),"send-time":datetime("2011-12-26T10:10:00"),"referred-topics":{{"sprint","voice-command"}},"message-text":" like sprint the voice-command is mind-blowing:)"}
+{"tweetid":"5","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("40.09,92.69"),"send-time":datetime("2006-08-04T10:10:00"),"referred-topics":{{"motorola","speed"}},"message-text":" can't stand motorola its speed is terrible:("}
+{"tweetid":"6","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("47.51,83.99"),"send-time":datetime("2010-05-07T10:10:00"),"referred-topics":{{"iphone","voice-clarity"}},"message-text":" like iphone the voice-clarity is good:)"}
+{"tweetid":"7","user":{"screen-name":"ChangEwing_573","lang":"en","friends_count":182,"statuses_count":394,"name":"Chang Ewing","followers_count":32136},"sender-location":point("36.21,72.6"),"send-time":datetime("2011-08-25T10:10:00"),"referred-topics":{{"samsung","platform"}},"message-text":" like samsung the platform is good"}
+{"tweetid":"8","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("46.05,93.34"),"send-time":datetime("2005-10-14T10:10:00"),"referred-topics":{{"t-mobile","shortcut-menu"}},"message-text":" like t-mobile the shortcut-menu is awesome:)"}
+{"tweetid":"9","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("36.86,74.62"),"send-time":datetime("2012-07-21T10:10:00"),"referred-topics":{{"verizon","voicemail-service"}},"message-text":" love verizon its voicemail-service is awesome"}
+{"tweetid":"10","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("29.15,76.53"),"send-time":datetime("2008-01-26T10:10:00"),"referred-topics":{{"verizon","voice-clarity"}},"message-text":" hate verizon its voice-clarity is OMG:("}
+{"tweetid":"11","user":{"screen-name":"NilaMilliron_tw","lang":"en","friends_count":445,"statuses_count":164,"name":"Nila Milliron","followers_count":22649},"sender-location":point("37.59,68.42"),"send-time":datetime("2008-03-09T10:10:00"),"referred-topics":{{"iphone","platform"}},"message-text":" can't stand iphone its platform is terrible"}
+{"tweetid":"12","user":{"screen-name":"OliJackson_512","lang":"en","friends_count":445,"statuses_count":164,"name":"Oli Jackson","followers_count":22649},"sender-location":point("24.82,94.63"),"send-time":datetime("2010-02-13T10:10:00"),"referred-topics":{{"samsung","voice-command"}},"message-text":" like samsung the voice-command is amazing:)"}
diff --git a/asterix-examples/src/main/resources/tiny-social/data/twu.adm b/asterix-examples/src/main/resources/tiny-social/data/twu.adm
new file mode 100644
index 0000000..32a1917
--- /dev/null
+++ b/asterix-examples/src/main/resources/tiny-social/data/twu.adm
@@ -0,0 +1,4 @@
+{"screen-name":"NathanGiesen@211","lang":"en","friends_count":18,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416}
+{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159}
+{"screen-name":"NilaMilliron_tw","lang":"en","friends_count":445,"statuses_count":164,"name":"Nila Milliron","followers_count":22649}
+{"screen-name":"ChangEwing_573","lang":"en","friends_count":182,"statuses_count":394,"name":"Chang Ewing","followers_count":32136}
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index 2e99b7c..aff6967 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -71,9 +71,7 @@
 		<dependency>
 			<groupId>javax.servlet</groupId>
 			<artifactId>servlet-api</artifactId>
-			<version>2.5</version>
 			<type>jar</type>
-			<scope>compile</scope>
 		</dependency>
 		<dependency>
 			<groupId>junit</groupId>
@@ -119,6 +117,12 @@
 			<version>1.0.0</version>
 			<type>jar</type>
 			<scope>compile</scope>
+			<exclusions>
+				<exclusion>
+					<artifactId>rome</artifactId>
+					<groupId>net.java.dev.rome</groupId>
+				</exclusion>
+			</exclusions>
 		</dependency>
 		<dependency>
 			<groupId>rome</groupId>
@@ -127,7 +131,8 @@
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-dataflow-hadoop</artifactId>
+			<artifactId>hyracks-hdfs-core</artifactId>
+			<version>${hyracks.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>jdom</groupId>
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
index f1f5884..8c8880a 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/CNNFeedAdapterFactory.java
@@ -23,9 +23,10 @@
  * A factory class for creating the @see {CNNFeedAdapter}.  
  */
 public class CNNFeedAdapterFactory implements ITypedDatasetAdapterFactory {
+    private static final long serialVersionUID = 1L;
 
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, String> configuration) throws Exception {
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception {
         CNNFeedAdapter cnnFeedAdapter = new CNNFeedAdapter();
         cnnFeedAdapter.configure(configuration);
         return cnnFeedAdapter;
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
index 6fcb710..c267658 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
@@ -14,22 +14,79 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
+import java.util.Arrays;
+import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+
 import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
+import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
+import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
 
 /**
  * A factory class for creating an instance of HDFSAdapter
  */
+@SuppressWarnings("deprecation")
 public class HDFSAdapterFactory implements IGenericDatasetAdapterFactory {
+    private static final long serialVersionUID = 1L;
 
     public static final String HDFS_ADAPTER_NAME = "hdfs";
+    public static final String CLUSTER_LOCATIONS = "cluster-locations";
+    public static transient String SCHEDULER = "hdfs-scheduler";
+
+    public static final String KEY_HDFS_URL = "hdfs";
+    public static final String KEY_PATH = "path";
+    public static final String KEY_INPUT_FORMAT = "input-format";
+    public static final String INPUT_FORMAT_TEXT = "text-input-format";
+    public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
+
+    private transient AlgebricksPartitionConstraint clusterLocations;
+    private String[] readSchedule;
+    private boolean executed[];
+    private InputSplitsFactory inputSplitsFactory;
+    private ConfFactory confFactory;
+    private boolean setup = false;
+
+    private static final Map<String, String> formatClassNames = initInputFormatMap();
+
+    private static Map<String, String> initInputFormatMap() {
+        Map<String, String> formatClassNames = new HashMap<String, String>();
+        formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
+        formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
+        return formatClassNames;
+    }
 
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, String> configuration, IAType atype) throws Exception {
-        HDFSAdapter hdfsAdapter = new HDFSAdapter(atype);
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+        if (!setup) {
+            /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+            configureJobConf(configuration);
+            JobConf conf = configureJobConf(configuration);
+            confFactory = new ConfFactory(conf);
+
+            clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+            int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+
+            InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+            inputSplitsFactory = new InputSplitsFactory(inputSplits);
+
+            Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+            readSchedule = scheduler.getLocationConstraints(inputSplits);
+            executed = new boolean[readSchedule.length];
+            Arrays.fill(executed, false);
+
+            setup = true;
+        }
+        JobConf conf = confFactory.getConf();
+        InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+        HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
         hdfsAdapter.configure(configuration);
         return hdfsAdapter;
     }
@@ -39,4 +96,15 @@
         return HDFS_ADAPTER_NAME;
     }
 
+    private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+        JobConf conf = new JobConf();
+        conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
+        conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+        conf.setClassLoader(HDFSAdapter.class.getClassLoader());
+        conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
+        conf.set("mapred.input.format.class",
+                (String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
+        return conf;
+    }
+
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
index 5e28eed..ba6ade5 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
@@ -14,20 +14,92 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
+import java.util.Arrays;
+import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
+import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
+import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
 
 /**
  * A factory class for creating an instance of HiveAdapter
  */
+@SuppressWarnings("deprecation")
 public class HiveAdapterFactory implements IGenericDatasetAdapterFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final String HDFS_ADAPTER_NAME = "hdfs";
+    public static final String CLUSTER_LOCATIONS = "cluster-locations";
+    public static transient String SCHEDULER = "hdfs-scheduler";
+
+    public static final String KEY_HDFS_URL = "hdfs";
+    public static final String KEY_PATH = "path";
+    public static final String KEY_INPUT_FORMAT = "input-format";
+    public static final String INPUT_FORMAT_TEXT = "text-input-format";
+    public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
+
+    public static final String KEY_FORMAT = "format";
+    public static final String KEY_PARSER_FACTORY = "parser";
+    public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+    public static final String FORMAT_ADM = "adm";
+
+    public static final String HIVE_DATABASE = "database";
+    public static final String HIVE_TABLE = "table";
+    public static final String HIVE_HOME = "hive-home";
+    public static final String HIVE_METASTORE_URI = "metastore-uri";
+    public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
+    public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
+
+    private String[] readSchedule;
+    private boolean executed[];
+    private InputSplitsFactory inputSplitsFactory;
+    private ConfFactory confFactory;
+    private transient AlgebricksPartitionConstraint clusterLocations;
+    private boolean setup = false;
+
+    private static final Map<String, String> formatClassNames = initInputFormatMap();
+
+    private static Map<String, String> initInputFormatMap() {
+        Map<String, String> formatClassNames = new HashMap<String, String>();
+        formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
+        formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
+        return formatClassNames;
+    }
 
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, String> configuration, IAType type) throws Exception {
-        HiveAdapter hiveAdapter = new HiveAdapter(type);
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+        if (!setup) {
+            /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+            configureJobConf(configuration);
+            JobConf conf = configureJobConf(configuration);
+            confFactory = new ConfFactory(conf);
+
+            clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+            int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+
+            InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+            inputSplitsFactory = new InputSplitsFactory(inputSplits);
+
+            Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+            readSchedule = scheduler.getLocationConstraints(inputSplits);
+            executed = new boolean[readSchedule.length];
+            Arrays.fill(executed, false);
+
+            setup = true;
+        }
+        JobConf conf = confFactory.getConf();
+        InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+        HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
         hiveAdapter.configure(configuration);
         return hiveAdapter;
     }
@@ -36,4 +108,37 @@
     public String getName() {
         return "hive";
     }
+
+    private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+        JobConf conf = new JobConf();
+
+        /** configure hive */
+        String database = (String) configuration.get(HIVE_DATABASE);
+        String tablePath = null;
+        if (database == null) {
+            tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + configuration.get(HIVE_TABLE);
+        } else {
+            tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
+                    + configuration.get(HIVE_TABLE);
+        }
+        configuration.put(HDFSAdapter.KEY_PATH, tablePath);
+        if (!configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT)) {
+            throw new IllegalArgumentException("format" + configuration.get(KEY_FORMAT) + " is not supported");
+        }
+
+        if (!(configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT) || configuration
+                .get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))) {
+            throw new IllegalArgumentException("file input format"
+                    + configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT) + " is not supported");
+        }
+
+        /** configure hdfs */
+        conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
+        conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+        conf.setClassLoader(HDFSAdapter.class.getClassLoader());
+        conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
+        conf.set("mapred.input.format.class",
+                (String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
+        return conf;
+    }
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IAdapterFactory.java
index 45fd6cf..697c8ea 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IAdapterFactory.java
@@ -14,12 +14,14 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
+import java.io.Serializable;
+
 /**
  * Base interface for IGenericDatasetAdapterFactory and ITypedDatasetAdapterFactory.
  * Acts as a marker interface indicating that the implementation provides functionality
  * for creating an adapter.
  */
-public interface IAdapterFactory {
+public interface IAdapterFactory extends Serializable {
 
     /**
      * Returns the display name corresponding to the Adapter type that is created by the factory.
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
index 093a3dd..e8d120a 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
@@ -38,6 +38,6 @@
      * @return An instance of IDatasourceAdapter.
      * @throws Exception
      */
-    public IDatasourceAdapter createAdapter(Map<String, String> configuration, IAType atype) throws Exception;
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception;
 
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/ITypedDatasetAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/ITypedDatasetAdapterFactory.java
index 0f9978e..84a5ca8 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/ITypedDatasetAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/ITypedDatasetAdapterFactory.java
@@ -33,6 +33,6 @@
      * @return An instance of IDatasourceAdapter.
      * @throws Exception
      */
-    public IDatasourceAdapter createAdapter(Map<String, String> configuration) throws Exception;
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception;
 
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
index 2040949..659fd23 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
@@ -26,11 +26,11 @@
  * an NC.
  */
 public class NCFileSystemAdapterFactory implements IGenericDatasetAdapterFactory {
-
+    private static final long serialVersionUID = 1L;
     public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
 
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, String> configuration, IAType atype) throws Exception {
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
         NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(atype);
         fsAdapter.configure(configuration);
         return fsAdapter;
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
index bc00469..e63be17 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedTwitterAdapterFactory.java
@@ -25,11 +25,11 @@
  * via pull-based Twitter API.
  */
 public class PullBasedTwitterAdapterFactory implements ITypedDatasetAdapterFactory {
-
+    private static final long serialVersionUID = 1L;
     public static final String PULL_BASED_TWITTER_ADAPTER_NAME = "pull_twitter";
 
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, String> configuration) throws Exception {
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception {
         PullBasedTwitterAdapter twitterAdapter = new PullBasedTwitterAdapter();
         twitterAdapter.configure(configuration);
         return twitterAdapter;
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
index bbbea38..3cd22e8 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/RSSFeedAdapterFactory.java
@@ -24,11 +24,11 @@
  * RSSFeedAdapter provides the functionality of fetching an RSS based feed.
  */
 public class RSSFeedAdapterFactory implements ITypedDatasetAdapterFactory {
-
+    private static final long serialVersionUID = 1L;
     public static final String RSS_FEED_ADAPTER_NAME = "rss_feed";
 
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, String> configuration) throws Exception {
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration) throws Exception {
         RSSFeedAdapter rssFeedAdapter = new RSSFeedAdapter();
         rssFeedAdapter.configure(configuration);
         return rssFeedAdapter;
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java
index fb4cc99..f9f72cf 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataScanOperatorDescriptor.java
@@ -19,8 +19,6 @@
 import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -38,61 +36,24 @@
 
     private static final long serialVersionUID = 1L;
 
-    private final String adapterFactory;
-    private final Map<String, String> adapterConfiguration;
+    private final Map<String, Object> adapterConfiguration;
     private final IAType atype;
     private IGenericDatasetAdapterFactory datasourceAdapterFactory;
 
-    public ExternalDataScanOperatorDescriptor(JobSpecification spec, String adapter, Map<String, String> arguments,
-            IAType atype, RecordDescriptor rDesc) {
+    public ExternalDataScanOperatorDescriptor(JobSpecification spec, Map<String, Object> arguments, IAType atype,
+            RecordDescriptor rDesc, IGenericDatasetAdapterFactory dataSourceAdapterFactory) {
         super(spec, 0, 1);
         recordDescriptors[0] = rDesc;
-        this.adapterFactory = adapter;
         this.adapterConfiguration = arguments;
         this.atype = atype;
-    }
-
-    @Override
-    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ICCApplicationContext appCtx) {
-
-        /*
-        Comment: The following code is commented out. This is because constraints are being set at compile time so that they can 
-        be propagated to upstream Asterix operators. Hyracks has to provide a way to propagate constraints to upstream operators.
-        Once it is there, we will uncomment the following code.  
-        
-        AlgebricksPartitionConstraint constraint = datasourceReadAdapter.getPartitionConstraint();
-        switch (constraint.getPartitionConstraintType()) {
-            case ABSOLUTE:
-                String[] locations = ((AlgebricksAbsolutePartitionConstraint) constraint).getLocations();
-                for (int i = 0; i < locations.length; ++i) {
-                    constraintAcceptor.addConstraint(new Constraint(new PartitionLocationExpression(this.odId, i),
-                            new ConstantExpression(locations[i])));
-                }
-                constraintAcceptor.addConstraint(new Constraint(new PartitionCountExpression(this.odId),
-                        new ConstantExpression(locations.length)));
-
-                break;
-            case COUNT:
-                constraintAcceptor.addConstraint(new Constraint(new PartitionCountExpression(this.odId),
-                        new ConstantExpression(((AlgebricksCountPartitionConstraint) constraint).getCount())));
-                break;
-            default:
-                throw new IllegalStateException(" Constraint type :" + constraint.getPartitionConstraintType()
-                        + " not supported");
-
-        }*/
-
+        this.datasourceAdapterFactory = dataSourceAdapterFactory;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
             throws HyracksDataException {
-        try {
-            datasourceAdapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactory).newInstance();
-        } catch (Exception e) {
-            throw new HyracksDataException("initialization of adapter failed", e);
-        }
+
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
             @Override
             public void initialize() throws HyracksDataException {
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorDescriptor.java
index 2da4e76..f07168a 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorDescriptor.java
@@ -35,52 +35,46 @@
  * Operator responsible for ingesting data from an external source. This
  * operator uses a (configurable) adapter associated with the feed dataset.
  */
-public class FeedIntakeOperatorDescriptor extends
-		AbstractSingleActivityOperatorDescriptor {
+public class FeedIntakeOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
 
-	private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 1L;
 
-	private final String adapterFactoryClassName;
-	private final Map<String, String> adapterConfiguration;
-	private final IAType atype;
-	private final FeedId feedId;
+    private final String adapterFactoryClassName;
+    private final Map<String, Object> adapterConfiguration;
+    private final IAType atype;
+    private final FeedId feedId;
+    private final IAdapterFactory datasourceAdapterFactory;
 
-	private transient IAdapterFactory datasourceAdapterFactory;
+    public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedId feedId, String adapter,
+            Map<String, Object> arguments, ARecordType atype, RecordDescriptor rDesc,
+            IAdapterFactory datasourceAdapterFactory) {
+        super(spec, 1, 1);
+        recordDescriptors[0] = rDesc;
+        this.adapterFactoryClassName = adapter;
+        this.adapterConfiguration = arguments;
+        this.atype = atype;
+        this.feedId = feedId;
+        this.datasourceAdapterFactory = datasourceAdapterFactory;
+    }
 
-	public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedId feedId,
-			String adapter, Map<String, String> arguments, ARecordType atype,
-			RecordDescriptor rDesc) {
-		super(spec, 1, 1);
-		recordDescriptors[0] = rDesc;
-		this.adapterFactoryClassName = adapter;
-		this.adapterConfiguration = arguments;
-		this.atype = atype;
-		this.feedId = feedId;
-	}
-
-	public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-			IRecordDescriptorProvider recordDescProvider, final int partition,
-			int nPartitions) throws HyracksDataException {
-		ITypedDatasourceAdapter adapter;
-		try {
-			datasourceAdapterFactory = (IAdapterFactory) Class.forName(
-					adapterFactoryClassName).newInstance();
-			if (datasourceAdapterFactory instanceof IGenericDatasetAdapterFactory) {
-				adapter = (ITypedDatasourceAdapter) ((IGenericDatasetAdapterFactory) datasourceAdapterFactory)
-						.createAdapter(adapterConfiguration, atype);
-			} else if (datasourceAdapterFactory instanceof ITypedDatasetAdapterFactory) {
-				adapter = (ITypedDatasourceAdapter) ((ITypedDatasetAdapterFactory) datasourceAdapterFactory)
-						.createAdapter(adapterConfiguration);
-			} else {
-				throw new IllegalStateException(
-						" Unknown adapter factory type for "
-								+ adapterFactoryClassName);
-			}
-			adapter.initialize(ctx);
-		} catch (Exception e) {
-			throw new HyracksDataException("initialization of adapter failed",
-					e);
-		}
-		return new FeedIntakeOperatorNodePushable(feedId, adapter, partition);
-	}
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+            throws HyracksDataException {
+        ITypedDatasourceAdapter adapter;
+        try {
+            if (datasourceAdapterFactory instanceof IGenericDatasetAdapterFactory) {
+                adapter = (ITypedDatasourceAdapter) ((IGenericDatasetAdapterFactory) datasourceAdapterFactory)
+                        .createAdapter(adapterConfiguration, atype);
+            } else if (datasourceAdapterFactory instanceof ITypedDatasetAdapterFactory) {
+                adapter = (ITypedDatasourceAdapter) ((ITypedDatasetAdapterFactory) datasourceAdapterFactory)
+                        .createAdapter(adapterConfiguration);
+            } else {
+                throw new IllegalStateException(" Unknown adapter factory type for " + adapterFactoryClassName);
+            }
+            adapter.initialize(ctx);
+        } catch (Exception e) {
+            throw new HyracksDataException("initialization of adapter failed", e);
+        }
+        return new FeedIntakeOperatorNodePushable(feedId, adapter, partition);
+    }
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorNodePushable.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorNodePushable.java
index d0dbb98..fd40b03 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorNodePushable.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/FeedIntakeOperatorNodePushable.java
@@ -17,6 +17,7 @@
 import java.nio.ByteBuffer;
 import java.util.concurrent.LinkedBlockingQueue;
 
+import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.external.feed.lifecycle.AlterFeedMessage;
 import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
@@ -51,7 +52,7 @@
     public void open() throws HyracksDataException {
         if (adapter instanceof IManagedFeedAdapter) {
             feedInboxMonitor = new FeedInboxMonitor((IManagedFeedAdapter) adapter, inbox, partition);
-            feedInboxMonitor.start();
+            AsterixThreadExecutor.INSTANCE.execute(feedInboxMonitor);
             feedManager.registerFeedMsgQueue(feedId, inbox);
         }
         writer.open();
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AbstractDatasourceAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AbstractDatasourceAdapter.java
index 440ee8c..23e545d 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AbstractDatasourceAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/AbstractDatasourceAdapter.java
@@ -36,7 +36,7 @@
 
     private static final long serialVersionUID = 1L;
 
-    protected Map<String, String> configuration;
+    protected Map<String, Object> configuration;
     protected transient AlgebricksPartitionConstraint partitionConstraint;
     protected IAType atype;
     protected IHyracksTaskContext ctx;
@@ -51,15 +51,15 @@
         typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
     }
 
-    protected static final Map<String, String> formatToParserFactoryMap = initializeFormatParserFactoryMap();
+    protected static final Map<String, Object> formatToParserFactoryMap = initializeFormatParserFactoryMap();
 
     public static final String KEY_FORMAT = "format";
     public static final String KEY_PARSER_FACTORY = "parser";
     public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
     public static final String FORMAT_ADM = "adm";
 
-    private static Map<String, String> initializeFormatParserFactoryMap() {
-        Map<String, String> map = new HashMap<String, String>();
+    private static Map<String, Object> initializeFormatParserFactoryMap() {
+        Map<String, Object> map = new HashMap<String, Object>();
         map.put(FORMAT_DELIMITED_TEXT, "edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory");
         map.put(FORMAT_ADM, "edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory");
         return map;
@@ -77,7 +77,7 @@
      * @param attribute
      *            The attribute whose value needs to be obtained.
      */
-    public String getAdapterProperty(String attribute) {
+    public Object getAdapterProperty(String attribute) {
         return configuration.get(attribute);
     }
 
@@ -86,7 +86,7 @@
      * 
      * @return A Map<String,String> instance representing the adapter configuration.
      */
-    public Map<String, String> getConfiguration() {
+    public Map<String, Object> getConfiguration() {
         return configuration;
     }
 
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java
index 3898f7e..8976f7a 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/CNNFeedAdapter.java
@@ -70,9 +70,9 @@
     }
 
     @Override
-    public void configure(Map<String, String> arguments) throws Exception {
+    public void configure(Map<String, Object> arguments) throws Exception {
         configuration = arguments;
-        String rssURLProperty = configuration.get(KEY_RSS_URL);
+        String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
         if (rssURLProperty == null) {
             throw new IllegalArgumentException("no rss url provided");
         }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
index 9f8cedc..8ab252d 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
@@ -69,7 +69,7 @@
     public abstract void initialize(IHyracksTaskContext ctx) throws Exception;
 
     @Override
-    public abstract void configure(Map<String, String> arguments) throws Exception;
+    public abstract void configure(Map<String, Object> arguments) throws Exception;
 
     @Override
     public abstract AdapterType getAdapterType();
@@ -82,14 +82,14 @@
     }
 
     protected void configureFormat() throws Exception {
-        String parserFactoryClassname = configuration.get(KEY_PARSER_FACTORY);
+        String parserFactoryClassname = (String) configuration.get(KEY_PARSER_FACTORY);
         if (parserFactoryClassname == null) {
-            String specifiedFormat = configuration.get(KEY_FORMAT);
+            String specifiedFormat = (String) configuration.get(KEY_FORMAT);
             if (specifiedFormat == null) {
                 throw new IllegalArgumentException(" Unspecified data format");
             } else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
                 parserFactory = getDelimitedDataTupleParserFactory((ARecordType) atype);
-            } else if (FORMAT_ADM.equalsIgnoreCase(configuration.get(KEY_FORMAT))) {
+            } else if (FORMAT_ADM.equalsIgnoreCase((String)configuration.get(KEY_FORMAT))) {
                 parserFactory = getADMDataTupleParserFactory((ARecordType) atype);
             } else {
                 throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
index 1e05b2f..02eacf5 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
@@ -14,18 +14,9 @@
  */
 package edu.uci.ics.asterix.external.dataset.adapter;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Counters.Counter;
@@ -37,14 +28,9 @@
 import org.apache.hadoop.mapred.TextInputFormat;
 
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.om.util.AsterixRuntimeUtil;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.hadoop.util.InputSplitsProxy;
 
 /**
  * Provides functionality for fetching external data stored in an HDFS instance.
@@ -53,118 +39,29 @@
 public class HDFSAdapter extends FileSystemBasedAdapter {
 
     private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(HDFSAdapter.class.getName());
 
-    public static final String KEY_HDFS_URL = "hdfs";
-    public static final String KEY_INPUT_FORMAT = "input-format";
-    public static final String INPUT_FORMAT_TEXT = "text-input-format";
-    public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
-
-    private Object[] inputSplits;
+    private transient String[] readSchedule;
+    private transient boolean executed[];
+    private transient InputSplit[] inputSplits;
     private transient JobConf conf;
-    private InputSplitsProxy inputSplitsProxy;
-    private static final Map<String, String> formatClassNames = initInputFormatMap();
+    private transient AlgebricksPartitionConstraint clusterLocations;
 
-    private static Map<String, String> initInputFormatMap() {
-        Map<String, String> formatClassNames = new HashMap<String, String>();
-        formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
-        formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
-        return formatClassNames;
-    }
+    private transient String nodeName;
 
-    public HDFSAdapter(IAType atype) {
+    public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
+            AlgebricksPartitionConstraint clusterLocations) {
         super(atype);
+        this.readSchedule = readSchedule;
+        this.executed = executed;
+        this.inputSplits = inputSplits;
+        this.conf = conf;
+        this.clusterLocations = clusterLocations;
     }
 
     @Override
-    public void configure(Map<String, String> arguments) throws Exception {
-        configuration = arguments;
+    public void configure(Map<String, Object> arguments) throws Exception {
+        this.configuration = arguments;
         configureFormat();
-        configureJobConf();
-        configureSplits();
-    }
-
-    private void configureSplits() throws IOException {
-        if (inputSplitsProxy == null) {
-            inputSplits = conf.getInputFormat().getSplits(conf, 0);
-        }
-        inputSplitsProxy = new InputSplitsProxy(conf, inputSplits);
-    }
-
-    private void configurePartitionConstraint() throws Exception {
-        List<String> locations = new ArrayList<String>();
-        Random random = new Random();
-        boolean couldConfigureLocationConstraints = false;
-        try {
-            Map<String, Set<String>> nodeControllers = AsterixRuntimeUtil.getNodeControllerMap();
-            for (Object inputSplit : inputSplits) {
-                String[] dataNodeLocations = ((InputSplit) inputSplit).getLocations();
-                if (dataNodeLocations == null || dataNodeLocations.length == 0) {
-                    throw new IllegalArgumentException("No datanode locations found: check hdfs path");
-                }
-
-                // loop over all replicas until a split location coincides
-                // with an asterix datanode location
-                for (String datanodeLocation : dataNodeLocations) {
-                    Set<String> nodeControllersAtLocation = null;
-                    try {
-                        nodeControllersAtLocation = nodeControllers.get(AsterixRuntimeUtil
-                                .getIPAddress(datanodeLocation));
-                    } catch (UnknownHostException uhe) {
-                        if (LOGGER.isLoggable(Level.WARNING)) {
-                            LOGGER.log(Level.WARNING, "Unknown host :" + datanodeLocation);
-                        }
-                        continue;
-                    }
-                    if (nodeControllersAtLocation == null || nodeControllersAtLocation.size() == 0) {
-                        if (LOGGER.isLoggable(Level.WARNING)) {
-                            LOGGER.log(Level.WARNING, "No node controller found at " + datanodeLocation
-                                    + " will look at replica location");
-                        }
-                        couldConfigureLocationConstraints = false;
-                    } else {
-                        int locationIndex = random.nextInt(nodeControllersAtLocation.size());
-                        String chosenLocation = (String) nodeControllersAtLocation.toArray()[locationIndex];
-                        locations.add(chosenLocation);
-                        if (LOGGER.isLoggable(Level.INFO)) {
-                            LOGGER.log(Level.INFO, "split : " + inputSplit + " to be processed by :" + chosenLocation);
-                        }
-                        couldConfigureLocationConstraints = true;
-                        break;
-                    }
-                }
-
-                /* none of the replica locations coincides with an Asterix
-                   node controller location.
-                */
-                if (!couldConfigureLocationConstraints) {
-                    List<String> allNodeControllers = AsterixRuntimeUtil.getAllNodeControllers();
-                    int locationIndex = random.nextInt(allNodeControllers.size());
-                    String chosenLocation = allNodeControllers.get(locationIndex);
-                    locations.add(chosenLocation);
-                    if (LOGGER.isLoggable(Level.SEVERE)) {
-                        LOGGER.log(Level.SEVERE, "No local node controller found to process split : " + inputSplit
-                                + " will be processed by a remote node controller:" + chosenLocation);
-                    }
-                }
-            }
-            partitionConstraint = new AlgebricksAbsolutePartitionConstraint(locations.toArray(new String[] {}));
-        } catch (Exception e) {
-            if (LOGGER.isLoggable(Level.SEVERE)) {
-                LOGGER.log(Level.SEVERE, "Encountered exception :" + e + " using count constraints");
-            }
-            partitionConstraint = new AlgebricksCountPartitionConstraint(inputSplits.length);
-        }
-    }
-
-    private JobConf configureJobConf() throws Exception {
-        conf = new JobConf();
-        conf.set("fs.default.name", configuration.get(KEY_HDFS_URL).trim());
-        conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
-        conf.setClassLoader(HDFSAdapter.class.getClassLoader());
-        conf.set("mapred.input.dir", configuration.get(KEY_PATH).trim());
-        conf.set("mapred.input.format.class", formatClassNames.get(configuration.get(KEY_INPUT_FORMAT).trim()));
-        return conf;
     }
 
     public AdapterType getAdapterType() {
@@ -174,7 +71,7 @@
     @Override
     public void initialize(IHyracksTaskContext ctx) throws Exception {
         this.ctx = ctx;
-        inputSplits = inputSplitsProxy.toInputSplits(conf);
+        this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
     }
 
     private Reporter getReporter() {
@@ -215,98 +112,124 @@
         return reporter;
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public InputStream getInputStream(int partition) throws IOException {
-        try {
-            InputStream inputStream;
-            if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
-                SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
-                RecordReader reader = format.getRecordReader(
-                        (org.apache.hadoop.mapred.FileSplit) inputSplits[partition], conf, getReporter());
-                inputStream = new HDFSStream(reader, ctx);
-            } else {
-                try {
+
+        return new InputStream() {
+
+            private RecordReader<Object, Text> reader;
+            private Object key;
+            private Text value;
+            private boolean hasMore = false;
+            private int EOL = "\n".getBytes()[0];
+            private Text pendingValue = null;
+            private int currentSplitIndex = 0;
+
+            @SuppressWarnings("unchecked")
+            private boolean moveToNext() throws IOException {
+                for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+                    /**
+                     * read all the partitions scheduled to the current node
+                     */
+                    if (readSchedule[currentSplitIndex].equals(nodeName)) {
+                        /**
+                         * pick an unread split to read
+                         * synchronize among simultaneous partitions in the same machine
+                         */
+                        synchronized (executed) {
+                            if (executed[currentSplitIndex] == false) {
+                                executed[currentSplitIndex] = true;
+                            } else {
+                                continue;
+                            }
+                        }
+
+                        /**
+                         * read the split
+                         */
+                        reader = getRecordReader(currentSplitIndex);
+                        key = reader.createKey();
+                        value = (Text) reader.createValue();
+                        return true;
+                    }
+                }
+                return false;
+            }
+
+            @Override
+            public int read(byte[] buffer, int offset, int len) throws IOException {
+                if (reader == null) {
+                    if (!moveToNext()) {
+                        //nothing to read
+                        return -1;
+                    }
+                }
+
+                int numBytes = 0;
+                if (pendingValue != null) {
+                    System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
+                    buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
+                    numBytes += pendingValue.getLength() + 1;
+                    pendingValue = null;
+                }
+
+                while (numBytes < len) {
+                    hasMore = reader.next(key, value);
+                    if (!hasMore) {
+                        while (moveToNext()) {
+                            hasMore = reader.next(key, value);
+                            if (hasMore) {
+                                //move to the next non-empty split
+                                break;
+                            }
+                        }
+                    }
+                    if (!hasMore) {
+                        return (numBytes == 0) ? -1 : numBytes;
+                    }
+                    int sizeOfNextTuple = value.getLength() + 1;
+                    if (numBytes + sizeOfNextTuple > len) {
+                        // cannot add tuple to current buffer
+                        // but the reader has moved pass the fetched tuple
+                        // we need to store this for a subsequent read call.
+                        // and return this then.
+                        pendingValue = value;
+                        break;
+                    } else {
+                        System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+                        buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+                        numBytes += sizeOfNextTuple;
+                    }
+                }
+                return numBytes;
+            }
+
+            @Override
+            public int read() throws IOException {
+                throw new NotImplementedException("Use read(byte[], int, int");
+            }
+
+            private RecordReader getRecordReader(int slitIndex) throws IOException {
+                if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
+                    SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
+                    RecordReader reader = format.getRecordReader(
+                            (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+                    return reader;
+                } else {
                     TextInputFormat format = (TextInputFormat) conf.getInputFormat();
                     RecordReader reader = format.getRecordReader(
-                            (org.apache.hadoop.mapred.FileSplit) inputSplits[partition], conf, getReporter());
-                    inputStream = new HDFSStream(reader, ctx);
-                } catch (FileNotFoundException e) {
-                    throw new HyracksDataException(e);
+                            (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+                    return reader;
                 }
             }
-            return inputStream;
-        } catch (Exception e) {
-            throw new IOException(e);
-        }
+
+        };
 
     }
 
     @Override
     public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        if (partitionConstraint == null) {
-            configurePartitionConstraint();
-        }
-        return partitionConstraint;
-    }
-
-}
-
-class HDFSStream extends InputStream {
-
-    private RecordReader<Object, Text> reader;
-    private final Object key;
-    private final Text value;
-    private boolean hasMore = false;
-    private static final int EOL = "\n".getBytes()[0];
-    private Text pendingValue = null;
-
-    public HDFSStream(RecordReader<Object, Text> reader, IHyracksTaskContext ctx) throws Exception {
-        this.reader = reader;
-        key = reader.createKey();
-        try {
-            value = (Text) reader.createValue();
-        } catch (ClassCastException cce) {
-            throw new Exception("value is not of type org.apache.hadoop.io.Text"
-                    + " type not supported in sequence file format", cce);
-        }
-    }
-
-    @Override
-    public int read(byte[] buffer, int offset, int len) throws IOException {
-        int numBytes = 0;
-        if (pendingValue != null) {
-            System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
-            buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
-            numBytes += pendingValue.getLength() + 1;
-            pendingValue = null;
-        }
-
-        while (numBytes < len) {
-            hasMore = reader.next(key, value);
-            if (!hasMore) {
-                return (numBytes == 0) ? -1 : numBytes;
-            }
-            int sizeOfNextTuple = value.getLength() + 1;
-            if (numBytes + sizeOfNextTuple > len) {
-                // cannot add tuple to current buffer
-                // but the reader has moved pass the fetched tuple
-                // we need to store this for a subsequent read call.
-                // and return this then.
-                pendingValue = value;
-                break;
-            } else {
-                System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-                buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-                numBytes += sizeOfNextTuple;
-            }
-        }
-        return numBytes;
-    }
-
-    @Override
-    public int read() throws IOException {
-        throw new NotImplementedException("Use read(byte[], int, int");
+        return clusterLocations;
     }
 
 }
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
index 3731eba..5e48834 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
@@ -16,6 +16,9 @@
 
 import java.util.Map;
 
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
@@ -24,6 +27,7 @@
 /**
  * Provides the functionality of fetching data in form of ADM records from a Hive dataset.
  */
+@SuppressWarnings("deprecation")
 public class HiveAdapter extends AbstractDatasourceAdapter {
 
     private static final long serialVersionUID = 1L;
@@ -37,8 +41,9 @@
 
     private HDFSAdapter hdfsAdapter;
 
-    public HiveAdapter(IAType atype) {
-        this.hdfsAdapter = new HDFSAdapter(atype);
+    public HiveAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
+            AlgebricksPartitionConstraint clusterLocations) {
+        this.hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
         this.atype = atype;
     }
 
@@ -48,33 +53,9 @@
     }
 
     @Override
-    public void configure(Map<String, String> arguments) throws Exception {
-        configuration = arguments;
-        configureHadoopAdapter();
-    }
-
-    private void configureHadoopAdapter() throws Exception {
-        String database = configuration.get(HIVE_DATABASE);
-        String tablePath = null;
-        if (database == null) {
-            tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + configuration.get(HIVE_TABLE);
-        } else {
-            tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
-                    + configuration.get(HIVE_TABLE);
-        }
-        configuration.put(HDFSAdapter.KEY_PATH, tablePath);
-        if (!configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT)) {
-            throw new IllegalArgumentException("format" + configuration.get(KEY_FORMAT) + " is not supported");
-        }
-
-        if (!(configuration.get(HDFSAdapter.KEY_INPUT_FORMAT).equals(HDFSAdapter.INPUT_FORMAT_TEXT) || configuration
-                .get(HDFSAdapter.KEY_INPUT_FORMAT).equals(HDFSAdapter.INPUT_FORMAT_SEQUENCE))) {
-            throw new IllegalArgumentException("file input format" + configuration.get(HDFSAdapter.KEY_INPUT_FORMAT)
-                    + " is not supported");
-        }
-
-        hdfsAdapter = new HDFSAdapter(atype);
-        hdfsAdapter.configure(configuration);
+    public void configure(Map<String, Object> arguments) throws Exception {
+        this.configuration = arguments;
+        this.hdfsAdapter.configure(arguments);
     }
 
     @Override
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IDatasourceAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IDatasourceAdapter.java
index b0dc32f..031f34f 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IDatasourceAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IDatasourceAdapter.java
@@ -70,7 +70,7 @@
      * @return String the value corresponding to the configuration parameter
      *         represented by the key- attributeKey.
      */
-    public String getAdapterProperty(String propertyKey);
+    public Object getAdapterProperty(String propertyKey);
 
     /**
      * Configures the IDatasourceAdapter instance.
@@ -100,7 +100,7 @@
      *            providing all arguments as a set of (key,value) pairs. These
      *            arguments are put into the metadata.
      */
-    public void configure(Map<String, String> arguments) throws Exception;
+    public void configure(Map<String, Object> arguments) throws Exception;
 
     /**
      * Returns a list of partition constraints. A partition constraint can be a
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
index ef39d45..9abc92a 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
@@ -44,9 +44,9 @@
     }
 
     @Override
-    public void configure(Map<String, String> arguments) throws Exception {
+    public void configure(Map<String, Object> arguments) throws Exception {
         this.configuration = arguments;
-        String[] splits = arguments.get(KEY_PATH).split(",");
+        String[] splits = ((String) arguments.get(KEY_PATH)).split(",");
         configureFileSplits(splits);
         configureFormat();
     }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
index ebfbcad..66d9f98 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterAdapter.java
@@ -31,9 +31,8 @@
  */
 public class PullBasedTwitterAdapter extends PullBasedAdapter implements IManagedFeedAdapter {
 
-   
     private static final long serialVersionUID = 1L;
-    
+
     public static final String QUERY = "query";
     public static final String INTERVAL = "interval";
 
@@ -49,7 +48,7 @@
     }
 
     @Override
-    public void configure(Map<String, String> arguments) throws Exception {
+    public void configure(Map<String, Object> arguments) throws Exception {
         configuration = arguments;
         String[] fieldNames = { "id", "username", "location", "text", "timestamp" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
@@ -69,12 +68,12 @@
     }
 
     @Override
-    public void stop()  {
+    public void stop() {
         tweetClient.stop();
     }
 
     @Override
-    public void alter(Map<String, String> properties)  {
+    public void alter(Map<String, String> properties) {
         alterRequested = true;
         this.alteredParams = properties;
     }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
index 2a07472..06cddfd 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedTwitterFeedClient.java
@@ -63,8 +63,8 @@
         tupleFieldValues = new String[recordType.getFieldNames().length];
     }
 
-    public void initialize(Map<String, String> params) {
-        this.keywords = params.get(PullBasedTwitterAdapter.QUERY);
+    public void initialize(Map<String, Object> params) {
+        this.keywords = (String) params.get(PullBasedTwitterAdapter.QUERY);
         this.query = new Query(keywords);
         query.setRpp(100);
     }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java
index 611183c..ccd6516 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/RSSFeedAdapter.java
@@ -72,9 +72,9 @@
     }
 
     @Override
-    public void configure(Map<String, String> arguments) throws Exception {
+    public void configure(Map<String, Object> arguments) throws Exception {
         configuration = arguments;
-        String rssURLProperty = configuration.get(KEY_RSS_URL);
+        String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
         if (rssURLProperty == null) {
             throw new IllegalArgumentException("no rss url provided");
         }
@@ -94,7 +94,7 @@
     }
 
     protected void reconfigure(Map<String, String> arguments) {
-        String rssURLProperty = configuration.get(KEY_RSS_URL);
+        String rssURLProperty = (String) configuration.get(KEY_RSS_URL);
         if (rssURLProperty != null) {
             initializeFeedURLs(rssURLProperty);
         }
diff --git a/asterix-installer/.gitignore b/asterix-installer/.gitignore
new file mode 100644
index 0000000..19f2e00
--- /dev/null
+++ b/asterix-installer/.gitignore
@@ -0,0 +1,2 @@
+/target
+/target
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
index 814f3c9..32918a7 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
@@ -30,7 +30,7 @@
 
     @Override
     protected void execCommand() throws Exception {
-        InstallerDriver.initConfig();
+        InstallerDriver.initConfig(true);
         String instanceName = ((AlterConfig) config).name;
         InstallerUtil.validateAsterixInstanceExists(instanceName, State.INACTIVE);
         ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
index 261de22..a0014f6 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
@@ -35,7 +35,7 @@
 
     @Override
     protected void execCommand() throws Exception {
-        InstallerDriver.initConfig();
+        InstallerDriver.initConfig(true);
         String asterixInstanceName = ((BackupConfig) config).name;
         AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
         List<BackupInfo> backupInfo = instance.getBackupInfo();
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
index 01a409e..a99b530 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
@@ -39,7 +39,7 @@
 
     @Override
     protected void execCommand() throws Exception {
-        InstallerDriver.initConfig();
+        InstallerDriver.initConfig(true);
         ValidateCommand validateCommand = new ValidateCommand();
         boolean valid = validateCommand.validateCluster(((CreateConfig) config).clusterPath);
         if (!valid) {
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
index 2279d8f..4c2bb09 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
@@ -28,7 +28,7 @@
 
     @Override
     protected void execCommand() throws Exception {
-        InstallerDriver.initConfig();
+        InstallerDriver.initConfig(true);
         String asterixInstanceName = ((DeleteConfig) config).name;
         AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
         PatternCreator pc = new PatternCreator();
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
index 9306a56..66c6e77 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
@@ -31,7 +31,7 @@
 
     @Override
     protected void execCommand() throws Exception {
-        InstallerDriver.initConfig();
+        InstallerDriver.initConfig(true);
         String asterixInstanceName = ((DescribeConfig) config).name;
         boolean adminView = ((DescribeConfig) config).admin;
         if (asterixInstanceName != null) {
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
index 9ef925a..d4d9de3 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
@@ -30,7 +30,7 @@
 
     @Override
     protected void execCommand() throws Exception {
-        InstallerDriver.initConfig();
+        InstallerDriver.initConfig(true);
         String asterixInstanceName = ((RestoreConfig) config).name;
         AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
         int backupId = ((RestoreConfig) config).backupId;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
index b9dd23d..f38184d 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
@@ -22,7 +22,7 @@
 
     @Override
     protected void execCommand() throws Exception {
-        InstallerDriver.initConfig();
+        InstallerDriver.initConfig(false);
         ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
         lookupService.stopService(InstallerDriver.getConfiguration());
     }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
index 1180a4e..9d3cb64 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
@@ -33,7 +33,7 @@
 
     @Override
     protected void execCommand() throws Exception {
-        InstallerDriver.initConfig();
+        InstallerDriver.initConfig(true);
         String asterixInstanceName = ((StartConfig) config).name;
         AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
         InstallerUtil.createAsterixZip(instance);
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
index dfd8c5e..f78cfbc 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
@@ -35,7 +35,7 @@
 
     @Override
     protected void execCommand() throws Exception {
-        InstallerDriver.initConfig();
+        InstallerDriver.initConfig(true);
         String asterixInstanceName = ((StopConfig) config).name;
         AsterixInstance asterixInstance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName,
                 State.ACTIVE, State.UNUSABLE);
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
index 08249ee..a9ab53b 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
@@ -55,7 +55,7 @@
         return conf;
     }
 
-    public static void initConfig() throws Exception {
+    public static void initConfig(boolean ensureLookupServiceIsRunning) throws Exception {
         File configFile = new File(managixHome + File.separator + MANAGIX_CONF_XML);
         JAXBContext configCtx = JAXBContext.newInstance(Configuration.class);
         Unmarshaller unmarshaller = configCtx.createUnmarshaller();
@@ -63,7 +63,7 @@
         asterixZip = initBinary("asterix-server");
 
         ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
-        if (!lookupService.isRunning(conf)) {
+        if (ensureLookupServiceIsRunning && !lookupService.isRunning(conf)) {
             lookupService.startService(conf);
         }
     }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
index 2b884ef..abf0420 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
@@ -49,6 +49,7 @@
 
 import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
 import edu.uci.ics.asterix.common.configuration.Store;
+import edu.uci.ics.asterix.common.configuration.Coredump;
 import edu.uci.ics.asterix.event.driver.EventDriver;
 import edu.uci.ics.asterix.event.management.EventrixClient;
 import edu.uci.ics.asterix.event.management.EventUtil;
@@ -223,6 +224,14 @@
         }
         configuration.setStore(stores);
 
+        List<Coredump> coredump = new ArrayList<Coredump>();
+        String coredumpDir = null;
+        for (Node node : cluster.getNode()) {
+            coredumpDir = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
+            coredump.add(new Coredump(asterixInstanceName + "_" + node.getId(), coredumpDir));
+        }
+        configuration.setCoredump(coredump);
+
         File asterixConfDir = new File(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName);
         asterixConfDir.mkdirs();
 
diff --git a/asterix-installer/src/main/resources/conf/asterix-configuration.xml b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
index b6958c5..8394a96 100644
--- a/asterix-installer/src/main/resources/conf/asterix-configuration.xml
+++ b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
@@ -22,10 +22,11 @@
 	</property>
 
 	<property>
-		<name>storage.buffercache.numpages</name>
-		<value>1024</value>
-		<description>The number of pages allocated to the disk buffer cache.
-			(Default = "1024")
+		<name>storage.buffercache.size</name>
+		<value>33554432</value>
+		<description>The size of memory allocated to the disk buffer cache.
+			The value should be a multiple of the buffer cache page size(Default
+			= "33554432" // 32MB)
 		</description>
 	</property>
 
@@ -47,18 +48,18 @@
 
 	<property>
 		<name>storage.memorycomponent.numpages</name>
-		<value>4096</value>
+		<value>1024</value>
 		<description>The number of pages to allocate for a memory component.
-			(Default = 4096)
+			(Default = 1024)
 		</description>
 	</property>
 
 	<property>
 		<name>storage.memorycomponent.globalbudget</name>
-		<value>1073741824</value>
+		<value>536870192</value>
 		<description>The total size of memory in bytes that the sum of all
 			open memory
-			components cannot exceed. (Default = "1073741824" // 1GB)
+			components cannot exceed. (Default = "536870192" // 512MB)
 		</description>
 	</property>
 
@@ -79,6 +80,14 @@
 	</property>
 
 	<property>
+		<name>txn.log.directory</name>
+		<value>asterix_logs/</value>
+		<description>The directory location for transaction logs. (Default =
+			"asterix_logs/")
+		</description>
+	</property>
+
+	<property>
 		<name>txn.log.buffer.numpages</name>
 		<value>8</value>
 		<description>The number of in-memory log buffer pages. (Default = "8")
@@ -103,10 +112,17 @@
 	</property>
 
 	<property>
+		<name>txn.log.disksectorsize</name>
+		<value>4096</value>
+		<description>The size of a disk sector. (Default = "4096")
+		</description>
+	</property>
+
+	<property>
 		<name>txn.log.groupcommitinterval</name>
-		<value>200</value>
+		<value>1</value>
 		<description>The group commit wait time in milliseconds. (Default =
-			"200" // 2ms)
+			"10" // 0.1ms)
 		</description>
 	</property>
 
@@ -127,6 +143,14 @@
 	</property>
 
 	<property>
+		<name>txn.log.checkpoint.history</name>
+		<value>0</value>
+		<description>The number of old log partition files to keep before
+			discarding. (Default = "0")
+		</description>
+	</property>
+
+	<property>
 		<name>txn.lock.escalationthreshold</name>
 		<value>1000</value>
 		<description>The number of entity level locks that need to be acquired
@@ -137,25 +161,42 @@
 
 	<property>
 		<name>txn.lock.shrinktimer</name>
-		<value>120000</value>
+		<value>5000</value>
 		<description>The time in milliseconds to wait before deallocating
-			unused lock manager memory. (Default = "120000" // 120s)
+			unused lock manager memory. (Default = "5000" // 5s)
+		</description>
+	</property>
+
+	<property>
+		<name>txn.lock.timeout.waitthreshold</name>
+		<value>60000</value>
+		<description>The time in milliseconds to wait before labeling a
+			transaction which has been waiting for a lock timed-out. (Default =
+			"60000" // 60s)
+		</description>
+	</property>
+
+	<property>
+		<name>txn.lock.timeout.sweepthreshold</name>
+		<value>10000</value>
+		<description>The time in milliseconds the timeout thread waits between
+			sweeps to detect timed-out transactions. (Default = "10000" // 10s)
 		</description>
 	</property>
 
 	<property>
 		<name>compiler.sortmemory</name>
-		<value>536870912</value>
+		<value>33554432</value>
 		<description>The amount of memory in bytes given to sort operations.
-			(Default = "536870912" // 512mb)
+			(Default = "33554432" // 32mb)
 		</description>
 	</property>
 
 	<property>
 		<name>compiler.joinmemory</name>
-		<value>536870912</value>
+		<value>33554432</value>
 		<description>The amount of memory in bytes given to join operations.
-			(Default = "536870912" // 512mb)
+			(Default = "33554432" // 32mb)
 		</description>
 	</property>
 
@@ -176,8 +217,8 @@
 
 	<property>
 		<name>api.port</name>
-		<value>19101</value>
-		<description>The port for the ASTERIX API server. (Default = 19101)
+		<value>19002</value>
+		<description>The port for the ASTERIX API server. (Default = 19002)
 		</description>
 	</property>
 
diff --git a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java
index dc6f643..c7beb28 100644
--- a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java
+++ b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixInstallerIntegrationUtil.java
@@ -87,7 +87,7 @@
         cmdHandler.processCommand(command.split(" "));
 
         startZookeeper();
-        InstallerDriver.initConfig();
+        InstallerDriver.initConfig(true);
         createInstance();
         hcc = new HyracksConnection(CC_IP_ADDRESS, DEFAULT_HYRACKS_CC_CLIENT_PORT);
     }
diff --git a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java
index 98c6bf0..de73c08 100644
--- a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java
+++ b/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixLifecycleIT.java
@@ -34,11 +34,18 @@
         AsterixInstallerIntegrationUtil.init();
         TestCaseContext.Builder b = new TestCaseContext.Builder();
         testCaseCollection = b.build(new File(PATH_BASE));
+        File outdir = new File(PATH_ACTUAL);
+        outdir.mkdirs();
     }
 
     @AfterClass
     public static void tearDown() throws Exception {
         AsterixInstallerIntegrationUtil.deinit();
+        File outdir = new File(PATH_ACTUAL);
+        File[] files = outdir.listFiles();
+        if (files == null || files.length == 0) {
+            outdir.delete();
+        }
     }
 
     @Parameters
diff --git a/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml b/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml
index 6b444e8..6be403e 100644
--- a/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml
+++ b/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml
@@ -18,8 +18,8 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
 	  <fork>true</fork>
         </configuration>
       </plugin>
diff --git a/asterix-metadata/pom.xml b/asterix-metadata/pom.xml
index 425e8ff..6f70568 100644
--- a/asterix-metadata/pom.xml
+++ b/asterix-metadata/pom.xml
@@ -42,6 +42,12 @@
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-runtime</artifactId>
+			<version>0.0.6-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-core</artifactId>
 			<version>0.20.2</version>
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
index 0382315..33bb549 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
@@ -22,8 +22,8 @@
 
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
-import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.Function;
@@ -220,19 +220,19 @@
             }
         }
     }
-    
+
     public Object dropIndex(Index index) {
         synchronized (indexes) {
             Map<String, Map<String, Index>> datasetMap = indexes.get(index.getDataverseName());
             if (datasetMap == null) {
                 return null;
             }
-            
+
             Map<String, Index> indexMap = datasetMap.get(index.getDatasetName());
             if (indexMap == null) {
                 return null;
             }
-            
+
             return indexMap.remove(index.getIndexName());
         }
     }
@@ -268,7 +268,7 @@
             return m.get(datasetName);
         }
     }
-    
+
     public Index getIndex(String dataverseName, String datasetName, String indexName) {
         synchronized (indexes) {
             Map<String, Map<String, Index>> datasetMap = indexes.get(dataverseName);
@@ -376,15 +376,13 @@
 
     public Object addAdapterIfNotExists(DatasourceAdapter adapter) {
         synchronized (adapters) {
-            DatasourceAdapter adapterObject = adapters.get(adapter.getAdapterIdentifier().getNamespace()).get(
-                    adapter.getAdapterIdentifier().getAdapterName());
-            if (adapterObject != null) {
-                Map<String, DatasourceAdapter> adaptersInDataverse = adapters.get(adapter.getAdapterIdentifier()
-                        .getNamespace());
-                if (adaptersInDataverse == null) {
-                    adaptersInDataverse = new HashMap<String, DatasourceAdapter>();
-                    adapters.put(adapter.getAdapterIdentifier().getNamespace(), adaptersInDataverse);
-                }
+            Map<String, DatasourceAdapter> adaptersInDataverse = adapters.get(adapter.getAdapterIdentifier().getNamespace());
+            if (adaptersInDataverse == null) {
+                adaptersInDataverse = new HashMap<String, DatasourceAdapter>();
+                adapters.put(adapter.getAdapterIdentifier().getNamespace(), adaptersInDataverse);
+            }
+            DatasourceAdapter adapterObject = adaptersInDataverse.get(adapter.getAdapterIdentifier().getAdapterName());
+            if (adapterObject == null) {
                 return adaptersInDataverse.put(adapter.getAdapterIdentifier().getAdapterName(), adapter);
             }
             return null;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
index 09ec2f7..c33d130 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
@@ -21,7 +21,9 @@
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
 import edu.uci.ics.asterix.metadata.api.IMetadataManager;
 import edu.uci.ics.asterix.metadata.api.IMetadataNode;
@@ -33,8 +35,6 @@
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
 
 /**
@@ -43,38 +43,29 @@
  * received from the metadata node, to avoid contacting the metadata node
  * repeatedly. We assume that this metadata manager is the only metadata manager
  * in an Asterix cluster. Therefore, no separate cache-invalidation mechanism is
- * needed at this point.
- * Assumptions/Limitations:
- * The metadata subsystem is started during NC Bootstrap start, i.e., when
- * Asterix is deployed.
- * The metadata subsystem is destroyed in NC Bootstrap end, i.e., when Asterix
- * is undeployed.
- * The metadata subsystem consists of the MetadataManager and the MatadataNode.
- * The MetadataManager provides users access to the metadata.
- * The MetadataNode implements direct access to the storage layer on behalf of
- * the MetadataManager, and translates the binary representation of ADM into
- * Java objects for consumption by the MetadataManager's users.
- * There is exactly one instance of the MetadataManager and of the MetadataNode
- * in the cluster, which may or may not be co-located on the same machine (or in
- * the same JVM).
- * The MetadataManager exists in the same JVM as its user's (e.g., the query
- * compiler).
- * The MetadataNode exists in the same JVM as it's transactional components
- * (LockManager, LogManager, etc.)
- * Users shall access the metadata only through the MetadataManager, and never
- * via the MetadataNode directly.
+ * needed at this point. Assumptions/Limitations: The metadata subsystem is
+ * started during NC Bootstrap start, i.e., when Asterix is deployed. The
+ * metadata subsystem is destroyed in NC Bootstrap end, i.e., when Asterix is
+ * undeployed. The metadata subsystem consists of the MetadataManager and the
+ * MatadataNode. The MetadataManager provides users access to the metadata. The
+ * MetadataNode implements direct access to the storage layer on behalf of the
+ * MetadataManager, and translates the binary representation of ADM into Java
+ * objects for consumption by the MetadataManager's users. There is exactly one
+ * instance of the MetadataManager and of the MetadataNode in the cluster, which
+ * may or may not be co-located on the same machine (or in the same JVM). The
+ * MetadataManager exists in the same JVM as its user's (e.g., the query
+ * compiler). The MetadataNode exists in the same JVM as it's transactional
+ * components (LockManager, LogManager, etc.) Users shall access the metadata
+ * only through the MetadataManager, and never via the MetadataNode directly.
  * Multiple threads may issue requests to the MetadataManager concurrently. For
  * the sake of accessing metadata, we assume a transaction consists of one
- * thread.
- * Users are responsible for locking the metadata (using the MetadataManager
- * API) before issuing requests.
- * The MetadataNode is responsible for acquiring finer-grained locks on behalf
- * of requests from the MetadataManager. Currently, locks are acquired per
- * BTree, since the BTree does not acquire even finer-grained locks yet
- * internally.
- * The metadata can be queried with AQL DML like any other dataset, but can only
- * be changed with AQL DDL.
- * The transaction ids for metadata transactions must be unique across the
+ * thread. Users are responsible for locking the metadata (using the
+ * MetadataManager API) before issuing requests. The MetadataNode is responsible
+ * for acquiring finer-grained locks on behalf of requests from the
+ * MetadataManager. Currently, locks are acquired per BTree, since the BTree
+ * does not acquire even finer-grained locks yet internally. The metadata can be
+ * queried with AQL DML like any other dataset, but can only be changed with AQL
+ * DDL. The transaction ids for metadata transactions must be unique across the
  * cluster, i.e., metadata transaction ids shall never "accidentally" overlap
  * with transaction ids of regular jobs or other metadata transactions.
  */
@@ -220,7 +211,7 @@
 
     @Override
     public void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException {
-        // add dataset into metadataNode 
+        // add dataset into metadataNode
         try {
             metadataNode.addDataset(ctx.getJobId(), dataset);
         } catch (RemoteException e) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index 5bdf086..c19b413 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
@@ -19,11 +19,19 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
-import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.metadata.api.IMetadataIndex;
 import edu.uci.ics.asterix.metadata.api.IMetadataNode;
@@ -55,16 +63,9 @@
 import edu.uci.ics.asterix.om.base.AMutableString;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallback;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallback;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetIdFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext.TransactionType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -93,7 +94,7 @@
     private static final DatasetId METADATA_DATASET_ID = new DatasetId(MetadataPrimaryIndexes.METADATA_DATASET_ID);
 
     private IIndexLifecycleManager indexLifecycleManager;
-    private TransactionSubsystem transactionSubsystem;
+    private ITransactionSubsystem transactionSubsystem;
 
     public static final MetadataNode INSTANCE = new MetadataNode();
 
@@ -101,7 +102,7 @@
         super();
     }
 
-    public void initialize(AsterixAppRuntimeContext runtimeContext) {
+    public void initialize(IAsterixAppRuntimeContext runtimeContext) {
         this.transactionSubsystem = runtimeContext.getTransactionSubsystem();
         this.indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
     }
@@ -113,14 +114,14 @@
 
     @Override
     public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
-        TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
         transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
     }
 
     @Override
     public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
         try {
-            TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
             transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
         } catch (ACIDException e) {
             e.printStackTrace();
@@ -130,13 +131,13 @@
 
     @Override
     public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
-        TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
         transactionSubsystem.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
     }
 
     @Override
     public void unlock(JobId jobId) throws ACIDException, RemoteException {
-        TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
         transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
     }
 
@@ -265,14 +266,15 @@
         ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
         indexLifecycleManager.open(resourceID);
 
-        //prepare a Callback for logging
+        // prepare a Callback for logging
         IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
                 lsmIndex, IndexOperation.INSERT);
 
         IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
-        TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
         txnCtx.setTransactionType(TransactionType.READ_WRITE);
+        txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
 
         // TODO: fix exceptions once new BTree exception model is in hyracks.
         indexAccessor.insert(tuple);
@@ -282,7 +284,7 @@
 
     private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
             IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws Exception {
-        TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
 
         if (metadataIndex.isPrimaryIndex()) {
             return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
@@ -376,7 +378,8 @@
                         searchKey);
                 deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
             } catch (TreeIndexException tie) {
-                //ignore this exception and continue deleting all relevant artifacts. 
+                // ignore this exception and continue deleting all relevant
+                // artifacts.
             }
 
             // Delete entry from secondary index 'group'.
@@ -390,7 +393,8 @@
                             MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameSearchKey);
                     deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
                 } catch (TreeIndexException tie) {
-                    //ignore this exception and continue deleting all relevant artifacts.
+                    // ignore this exception and continue deleting all relevant
+                    // artifacts.
                 }
             }
             // Delete entry from secondary index 'type'.
@@ -402,7 +406,8 @@
                         MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
                 deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
             } catch (TreeIndexException tie) {
-                //ignore this exception and continue deleting all relevant artifacts.
+                // ignore this exception and continue deleting all relevant
+                // artifacts.
             }
 
             // Delete entry(s) from the 'indexes' dataset.
@@ -502,7 +507,8 @@
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'datatype' dataset.
             ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
-            // This call uses the secondary index on datatype. Get nested types before deleting entry from secondary index.
+            // This call uses the secondary index on datatype. Get nested types
+            // before deleting entry from secondary index.
             List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
             deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
             deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
@@ -568,13 +574,14 @@
         long resourceID = metadataIndex.getResourceID();
         ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
         indexLifecycleManager.open(resourceID);
-        //prepare a Callback for logging
+        // prepare a Callback for logging
         IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
                 lsmIndex, IndexOperation.DELETE);
         IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
-        TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
         txnCtx.setTransactionType(TransactionType.READ_WRITE);
+        txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
 
         indexAccessor.delete(tuple);
         indexLifecycleManager.close(resourceID);
@@ -822,7 +829,8 @@
         }
         try {
             // Delete entry from the 'function' dataset.
-            ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName());
+            ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
+                    + functionSignature.getArity());
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'function' dataset.
             ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
@@ -853,7 +861,7 @@
         return results.get(0);
     }
 
-    //Debugging Method
+    // Debugging Method
     public String printMetadata() {
 
         StringBuilder sb = new StringBuilder();
@@ -974,8 +982,8 @@
     public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException {
         int mostRecentDatasetId = MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID;
         long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
-        IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
         try {
+            IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
             indexLifecycleManager.open(resourceID);
             IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
@@ -991,6 +999,8 @@
             try {
                 while (rangeCursor.hasNext()) {
                     rangeCursor.next();
+                    ITupleReference ref = rangeCursor.getTuple();
+                    Dataset ds = valueExtractor.getValue(jobId, rangeCursor.getTuple());
                     datasetId = ((Dataset) valueExtractor.getValue(jobId, rangeCursor.getTuple())).getDatasetId();
                     if (mostRecentDatasetId < datasetId) {
                         mostRecentDatasetId = datasetId;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
index cbd37d6..c76fb16 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
@@ -18,6 +18,7 @@
 import java.util.ArrayList;
 
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
 import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
@@ -27,7 +28,6 @@
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 
 /**
  * Used to implement serializable transactions against the MetadataCache.
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java
index ca6ab08..efd8ecb 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java
@@ -17,16 +17,13 @@
 
 import java.util.List;
 
+import edu.uci.ics.asterix.common.transactions.DatasetId;
 import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 
 /**
  * Descriptor interface for a primary or secondary index on metadata datasets.
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
index f11144e..527ee91 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
@@ -18,19 +18,18 @@
 import java.rmi.RemoteException;
 import java.util.List;
 
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 
 /**
  * A metadata manager provides user access to Asterix metadata (e.g., types,
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
index e0b5e96..9ec233b 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
@@ -20,18 +20,18 @@
 import java.rmi.RemoteException;
 import java.util.List;
 
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 
 /**
  * A metadata node stores metadata in its local storage structures (currently
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java
index d8958dd..6994bdc 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java
@@ -17,8 +17,8 @@
 
 import java.io.IOException;
 
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index f3ccc9c..b4f4ee5 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -26,13 +26,17 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
-import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
-import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
-import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
 import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
 import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
 import edu.uci.ics.asterix.metadata.IDatasetDetails;
@@ -55,13 +59,10 @@
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
-import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceManagerRepository;
 import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -69,23 +70,18 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexFileNameUtil;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
 import edu.uci.ics.hyracks.storage.common.file.LocalResource;
-import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
 
 /**
  * Initializes the remote metadata storage facilities ("universe") using a
@@ -98,10 +94,8 @@
  */
 public class MetadataBootstrap {
     private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
-    private static final int DEFAULT_MEM_PAGE_SIZE = 32768;
-    private static final int DEFAULT_MEM_NUM_PAGES = 100;
 
-    private static AsterixAppRuntimeContext runtimeContext;
+    private static IAsterixAppRuntimeContext runtimeContext;
 
     private static IBufferCache bufferCache;
     private static IFileMapProvider fileMapProvider;
@@ -132,7 +126,7 @@
 
     public static void startUniverse(IAsterixPropertiesProvider asterixPropertiesProvider,
             INCApplicationContext ncApplicationContext, boolean isNewUniverse) throws Exception {
-        runtimeContext = (AsterixAppRuntimeContext) ncApplicationContext.getApplicationObject();
+        runtimeContext = (IAsterixAppRuntimeContext) ncApplicationContext.getApplicationObject();
         propertiesProvider = asterixPropertiesProvider;
 
         // Initialize static metadata objects, such as record types and metadata
@@ -214,6 +208,7 @@
                 //change the exception type to AbortFailureException
                 throw new MetadataException(e);
             }
+            throw e;
         }
     }
 
@@ -333,46 +328,47 @@
     }
 
     private static void enlistMetadataDataset(IMetadataIndex index, boolean create) throws Exception {
-        String filePath = metadataStore + File.separator + index.getFileNameRelativePath();
+        String filePath = IndexFileNameUtil.prepareFileName(
+                metadataStore + File.separator + index.getFileNameRelativePath(),
+                runtimeContext.getMetaDataIODeviceId());
         FileReference file = new FileReference(new File(filePath));
-        IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), DEFAULT_MEM_PAGE_SIZE,
-                DEFAULT_MEM_NUM_PAGES, new TransientFileMapManager());
+        IVirtualBufferCache virtualBufferCache = runtimeContext.getVirtualBufferCache(index.getDatasetId().getId());
         ITypeTraits[] typeTraits = index.getTypeTraits();
         IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
         int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
-        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(DEFAULT_MEM_NUM_PAGES,
-                metaDataFrameFactory);
         LSMBTree lsmBtree = null;
         long resourceID = -1;
+        AsterixRuntimeComponentsProvider rtcProvider = index.isPrimaryIndex() ? AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER
+                : AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER;
+        ILSMOperationTracker opTracker = index.isPrimaryIndex() ? runtimeContext.getLSMBTreeOperationTracker(index
+                .getDatasetId().getId()) : new BaseOperationTracker(LSMBTreeIOOperationCallbackFactory.INSTANCE);
         if (create) {
-            lsmBtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, bufferCache,
-                    fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
-                    runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(),
-                    runtimeContext.getLSMBTreeOperationTrackerFactory(), runtimeContext.getLSMIOScheduler(),
-                    AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER);
+            lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, bufferCache, fileMapProvider,
+                    typeTraits, comparatorFactories, bloomFilterKeyFields,
+                    runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(), opTracker,
+                    runtimeContext.getLSMIOScheduler(), rtcProvider, runtimeContext.getMetaDataIODeviceId());
             lsmBtree.create();
             resourceID = runtimeContext.getResourceIdFactory().createId();
-            indexLifecycleManager.register(resourceID, lsmBtree);
-
-            AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
             ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
                     comparatorFactories, bloomFilterKeyFields, index.isPrimaryIndex(),
-                    storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages());
+                    runtimeContext.getMetaDataIODeviceId(), index.getDatasetId().getId());
             ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                     localResourceMetadata, LocalResource.LSMBTreeResource);
             ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
-            localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, file.getFile()
-                    .getPath(), 0));
+            localResourceRepository.insert(
+                    localResourceFactory.createLocalResource(resourceID, file.getFile().getPath(), 0),
+                    runtimeContext.getMetaDataIODeviceId());
+            indexLifecycleManager.register(resourceID, lsmBtree);
         } else {
             resourceID = localResourceRepository.getResourceByName(file.getFile().getPath()).getResourceId();
             lsmBtree = (LSMBTree) indexLifecycleManager.getIndex(resourceID);
             if (lsmBtree == null) {
-                lsmBtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
-                        bufferCache, fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
+                lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, ioManager, file, bufferCache,
+                        fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
                         runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(),
-                        runtimeContext.getLSMBTreeOperationTrackerFactory(), runtimeContext.getLSMIOScheduler(),
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER);
+                        opTracker, runtimeContext.getLSMIOScheduler(),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        runtimeContext.getMetaDataIODeviceId());
                 indexLifecycleManager.register(resourceID, lsmBtree);
             }
         }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java
index c0ce030..8036cfb 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java
@@ -20,6 +20,7 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixRuntimeException;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryHashFunctionFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -27,17 +28,12 @@
 import edu.uci.ics.asterix.metadata.api.IMetadataIndex;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 
 /**
  * Descriptor for a primary or secondary index on metadata datasets.
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index 2fc3e59..5bd7960 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -32,7 +32,6 @@
     public static IMetadataIndex NODE_DATASET;
     public static IMetadataIndex NODEGROUP_DATASET;
     public static IMetadataIndex FUNCTION_DATASET;
-
     public static final int METADATA_DATASET_ID = 0;
     public static final int DATAVERSE_DATASET_ID = 1;
     public static final int DATASET_DATASET_ID = 2;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
index f79b9b7..5444ac2 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
@@ -22,7 +22,6 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.annotations.TypeDataGen;
-import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
 import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -38,6 +37,7 @@
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
index e4c7ba2..0ed3c78 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
@@ -17,7 +17,6 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
@@ -28,6 +27,7 @@
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.ListSet;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
@@ -219,7 +219,7 @@
                     if (n < 2) {
                         pp = new RandomPartitioningProperty(domain);
                     } else {
-                        Set<LogicalVariable> pvars = new HashSet<LogicalVariable>();
+                        Set<LogicalVariable> pvars = new ListSet<LogicalVariable>();
                         int i = 0;
                         for (LogicalVariable v : scanVariables) {
                             pvars.add(v);
@@ -239,7 +239,7 @@
                     if (n < 2) {
                         pp = new RandomPartitioningProperty(domain);
                     } else {
-                        Set<LogicalVariable> pvars = new HashSet<LogicalVariable>();
+                        Set<LogicalVariable> pvars = new ListSet<LogicalVariable>();
                         int i = 0;
                         for (LogicalVariable v : scanVariables) {
                             pvars.add(v);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index 2e45802..3993719 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -23,16 +23,21 @@
 import java.util.Map;
 import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
-import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
+import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
 import edu.uci.ics.asterix.common.context.TransactionSubsystemProvider;
+import edu.uci.ics.asterix.common.dataflow.AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor;
+import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
 import edu.uci.ics.asterix.common.parse.IParseFileSplitsDecl;
+import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
 import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
 import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
 import edu.uci.ics.asterix.external.adapter.factory.ITypedDatasetAdapterFactory;
@@ -64,6 +69,8 @@
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.asterix.runtime.base.AsterixTupleFilterFactory;
 import edu.uci.ics.asterix.runtime.formats.FormatUtils;
@@ -71,11 +78,11 @@
 import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexSearchOperationCallbackFactory;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallbackFactory;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -99,6 +106,7 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
+import edu.uci.ics.hyracks.api.context.ICCContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -117,6 +125,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
 import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
@@ -127,9 +136,7 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexInsertUpdateDeleteOperator;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
@@ -155,6 +162,7 @@
     private final AsterixStorageProperties storageProperties;
 
     private static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
+    private static Scheduler hdfsScheduler;
 
     public String getPropertyValue(String propertyName) {
         return config.get(propertyName);
@@ -176,6 +184,16 @@
         this.defaultDataverse = defaultDataverse;
         this.stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
         this.storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
+        ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
+        try {
+            if (hdfsScheduler == null) {
+                //set the singleton hdfs scheduler
+                hdfsScheduler = new Scheduler(ccContext.getClusterControllerInfo().getClientNetAddress(), ccContext
+                        .getClusterControllerInfo().getClientNetPort());
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
     }
 
     public void setJobId(JobId jobId) {
@@ -341,8 +359,8 @@
                 adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
             }
 
-            adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(datasetDetails.getProperties(),
-                    itemType);
+            adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
+                    wrapProperties(datasetDetails.getProperties()), itemType);
         } catch (AlgebricksException ae) {
             throw ae;
         } catch (Exception e) {
@@ -360,7 +378,7 @@
         RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
 
         ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec,
-                adapterFactoryClassname, datasetDetails.getProperties(), rt, scannerDesc);
+                wrapPropertiesEmpty(datasetDetails.getProperties()), rt, scannerDesc, adapterFactory);
 
         AlgebricksPartitionConstraint constraint;
         try {
@@ -417,21 +435,23 @@
                 adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
                 if (adapterFactoryClassname != null) {
                 } else {
-                    // adapterName has been provided as a fully qualified classname 
+                    // adapterName has been provided as a fully qualified
+                    // classname
                     adapterFactoryClassname = adapterName;
                 }
                 adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
             }
 
             if (adapterFactory instanceof ITypedDatasetAdapterFactory) {
-                adapter = ((ITypedDatasetAdapterFactory) adapterFactory).createAdapter(datasetDetails.getProperties());
+                adapter = ((ITypedDatasetAdapterFactory) adapterFactory).createAdapter(wrapProperties(datasetDetails
+                        .getProperties()));
                 adapterOutputType = ((ITypedDatasourceAdapter) adapter).getAdapterOutputType();
             } else if (adapterFactory instanceof IGenericDatasetAdapterFactory) {
                 String outputTypeName = datasetDetails.getProperties().get(IGenericDatasetAdapterFactory.KEY_TYPE_NAME);
                 adapterOutputType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(),
                         outputTypeName).getDatatype();
                 adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
-                        datasetDetails.getProperties(), adapterOutputType);
+                        wrapProperties(datasetDetails.getProperties()), adapterOutputType);
             } else {
                 throw new IllegalStateException(" Unknown factory type for " + adapterFactoryClassname);
             }
@@ -448,7 +468,8 @@
 
         FeedIntakeOperatorDescriptor feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedId(
                 dataset.getDataverseName(), dataset.getDatasetName()), adapterFactoryClassname,
-                datasetDetails.getProperties(), (ARecordType) adapterOutputType, feedDesc);
+                this.wrapPropertiesEmpty(datasetDetails.getProperties()), (ARecordType) adapterOutputType, feedDesc,
+                adapterFactory);
 
         AlgebricksPartitionConstraint constraint = null;
         try {
@@ -530,7 +551,7 @@
                 }
 
                 AqlMetadataImplConfig aqlMetadataImplConfig = (AqlMetadataImplConfig) implConfig;
-                TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+                ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
                 if (aqlMetadataImplConfig != null && aqlMetadataImplConfig.isInstantLock()) {
                     searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId,
                             primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
@@ -539,17 +560,16 @@
                             primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
                 }
             }
+            AsterixRuntimeComponentsProvider rtcProvider = isSecondary ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
+                    : AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER;
             BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
                     appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
                     typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
                     lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            storageProperties.getMemoryComponentPageSize(),
-                            storageProperties.getMemoryComponentNumPages(),
-                            storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
+                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), rtcProvider, rtcProvider,
+                            rtcProvider, rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()),
+                    retainInput, searchCallbackFactory);
+
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
 
         } catch (MetadataException me) {
@@ -612,13 +632,12 @@
                     appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
                     typeTraits, comparatorFactories, keyFields, new LSMRTreeDataflowHelperFactory(
                             valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
                                     nestedKeyType.getTypeTag(), comparatorFactories.length),
-                            storageProperties.getMemoryComponentPageSize(),
-                            storageProperties.getMemoryComponentNumPages(),
                             storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
 
@@ -767,19 +786,20 @@
                 numElementsHint = Long.parseLong(dataset.getHints().get("CARDINALITY"));
             }
 
-            //TODO
-            //figure out the right behavior of the bulkload and then give the right callback
-            //(ex. what's the expected behavior when there is an error during bulkload?)
+            // TODO
+            // figure out the right behavior of the bulkload and then give the
+            // right callback
+            // (ex. what's the expected behavior when there is an error during
+            // bulkload?)
             TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                     appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
                     splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
                     GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, numElementsHint, new LSMBTreeDataflowHelperFactory(
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            storageProperties.getMemoryComponentPageSize(),
-                            storageProperties.getMemoryComponentNumPages(),
+                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
                             storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
@@ -828,7 +848,7 @@
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
                     dataSource.getId().getDataverseName(), datasetName, indexName);
 
-            //prepare callback
+            // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             int datasetId = dataset.getDatasetId();
             int[] primaryKeyFields = new int[numKeys];
@@ -839,18 +859,18 @@
             PrimaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new PrimaryIndexModificationOperationCallbackFactory(
                     jobId, datasetId, primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE);
 
-            LSMTreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(
+            AsterixLSMTreeInsertDeleteOperatorDescriptor insertDeleteOp = new AsterixLSMTreeInsertDeleteOperatorDescriptor(
                     spec, recordDesc, appContext.getStorageManagerInterface(),
                     appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                     comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
-                    new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
-                                    .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                            storageProperties.getBloomFilterFalsePositiveRate()), null, modificationCallbackFactory);
+                    new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+                                    .getBloomFilterFalsePositiveRate()), null, modificationCallbackFactory, true);
 
-            return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
+            return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
                     splitsAndConstraint.second);
 
         } catch (MetadataException me) {
@@ -1021,7 +1041,7 @@
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
                     dataverseName, datasetName, indexName);
 
-            //prepare callback
+            // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             int datasetId = dataset.getDatasetId();
             int[] primaryKeyFields = new int[primaryKeys.size()];
@@ -1035,17 +1055,17 @@
             SecondaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new SecondaryIndexModificationOperationCallbackFactory(
                     jobId, datasetId, primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE);
 
-            LSMTreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(
+            AsterixLSMTreeInsertDeleteOperatorDescriptor btreeBulkLoad = new AsterixLSMTreeInsertDeleteOperatorDescriptor(
                     spec, recordDesc, appContext.getStorageManagerInterface(),
                     appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                     comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
-                    new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
-                                    .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                            storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
-                    modificationCallbackFactory);
+                    new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
+                            AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, storageProperties
+                                    .getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory,
+                    false);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
         } catch (MetadataException e) {
@@ -1136,7 +1156,8 @@
             tokenComparatorFactories[0] = NonTaggedFormatUtil.getTokenBinaryComparatorFactory(secondaryKeyType);
             tokenTypeTraits[0] = NonTaggedFormatUtil.getTokenTypeTrait(secondaryKeyType);
             if (isPartitioned) {
-                // The partitioning field is hardcoded to be a short *without* an Asterix type tag.
+                // The partitioning field is hardcoded to be a short *without*
+                // an Asterix type tag.
                 tokenComparatorFactories[1] = PointableBinaryComparatorFactory.of(ShortPointable.FACTORY);
                 tokenTypeTraits[1] = ShortPointable.TYPE_TRAITS;
             }
@@ -1147,7 +1168,7 @@
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
                     dataverseName, datasetName, indexName);
 
-            //prepare callback
+            // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             int datasetId = dataset.getDatasetId();
             int[] primaryKeyFields = new int[primaryKeys.size()];
@@ -1161,18 +1182,16 @@
             SecondaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new SecondaryIndexModificationOperationCallbackFactory(
                     jobId, datasetId, primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_INVERTED_INDEX);
 
-            LSMInvertedIndexInsertUpdateDeleteOperator insertDeleteOp = new LSMInvertedIndexInsertUpdateDeleteOperator(
+            AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor insertDeleteOp = new AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor(
                     spec, recordDesc, appContext.getStorageManagerInterface(), splitsAndConstraint.first,
                     appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
                     invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation, indexOp,
-                    new LSMInvertedIndexDataflowHelperFactory(
+                    new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER, storageProperties
-                                    .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
-                            storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
-                    modificationCallbackFactory);
+                                    .getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
                     splitsAndConstraint.second);
         } catch (MetadataException e) {
@@ -1242,7 +1261,7 @@
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
                     dataverseName, datasetName, indexName);
 
-            //prepare callback
+            // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             int datasetId = dataset.getDatasetId();
             int[] primaryKeyFields = new int[numPrimaryKeys];
@@ -1256,20 +1275,19 @@
             SecondaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new SecondaryIndexModificationOperationCallbackFactory(
                     jobId, datasetId, primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_RTREE);
 
-            LSMTreeIndexInsertUpdateDeleteOperatorDescriptor rtreeUpdate = new LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(
+            AsterixLSMTreeInsertDeleteOperatorDescriptor rtreeUpdate = new AsterixLSMTreeInsertDeleteOperatorDescriptor(
                     spec, recordDesc, appContext.getStorageManagerInterface(),
                     appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                     comparatorFactories, null, fieldPermutation, indexOp, new LSMRTreeDataflowHelperFactory(
                             valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
                                     nestedKeyType.getTypeTag(), comparatorFactories.length),
-                            storageProperties.getMemoryComponentPageSize(),
-                            storageProperties.getMemoryComponentNumPages(),
                             storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
-                    modificationCallbackFactory);
+                    modificationCallbackFactory, false);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
         } catch (MetadataException | IOException e) {
             throw new AlgebricksException(e);
@@ -1336,8 +1354,13 @@
                 continue;
             }
             for (int i = 0; i < nodeStores.length; i++) {
-                File f = new File(nodeStores[i] + File.separator + relPathFile);
-                splits.add(new FileSplit(node, new FileReference(f)));
+                int numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(node);
+                for (int j = 0; j < nodeStores.length; j++) {
+                    for (int k = 0; k < numIODevices; k++) {
+                        File f = new File(nodeStores[j] + File.separator + relPathFile);
+                        splits.add(new FileSplit(node, new FileReference(f), k));
+                    }
+                }
             }
         }
         return splits.toArray(new FileSplit[] {});
@@ -1366,9 +1389,17 @@
                     LOGGER.warning("Node " + nd + " has no stores.");
                     throw new AlgebricksException("Node " + nd + " has no stores.");
                 } else {
+                    int numIODevices;
+                    if (datasetDetails.getNodeGroupName().compareTo(MetadataConstants.METADATA_NODEGROUP_NAME) == 0) {
+                        numIODevices = 1;
+                    } else {
+                        numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nd);
+                    }
                     for (int j = 0; j < nodeStores.length; j++) {
-                        File f = new File(nodeStores[j] + File.separator + relPathFile);
-                        splitArray.add(new FileSplit(nd, new FileReference(f)));
+                        for (int k = 0; k < numIODevices; k++) {
+                            File f = new File(nodeStores[j] + File.separator + relPathFile);
+                            splitArray.add(new FileSplit(nd, new FileReference(f), k));
+                        }
                     }
                 }
             }
@@ -1415,21 +1446,6 @@
         return dataverseName + File.separator + fileName;
     }
 
-    public Pair<IFileSplitProvider, IFileSplitProvider> getInvertedIndexFileSplitProviders(
-            IFileSplitProvider splitProvider) {
-        int numSplits = splitProvider.getFileSplits().length;
-        FileSplit[] btreeSplits = new FileSplit[numSplits];
-        FileSplit[] invListsSplits = new FileSplit[numSplits];
-        for (int i = 0; i < numSplits; i++) {
-            String nodeName = splitProvider.getFileSplits()[i].getNodeName();
-            String path = splitProvider.getFileSplits()[i].getLocalFile().getFile().getPath();
-            btreeSplits[i] = new FileSplit(nodeName, path + "_$btree");
-            invListsSplits[i] = new FileSplit(nodeName, path + "_$invlists");
-        }
-        return new Pair<IFileSplitProvider, IFileSplitProvider>(new ConstantFileSplitProvider(btreeSplits),
-                new ConstantFileSplitProvider(invListsSplits));
-    }
-
     public Dataset findDataset(String dataverse, String dataset) throws AlgebricksException {
         try {
             return MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverse, dataset);
@@ -1461,10 +1477,13 @@
 
     public AlgebricksPartitionConstraint getClusterLocations() {
         ArrayList<String> locs = new ArrayList<String>();
-        for (String k : stores.keySet()) {
-            String[] nodeStores = stores.get(k);
+        for (String i : stores.keySet()) {
+            String[] nodeStores = stores.get(i);
+            int numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(i);
             for (int j = 0; j < nodeStores.length; j++) {
-                locs.add(k);
+                for (int k = 0; k < numIODevices; k++) {
+                    locs.add(i);
+                }
             }
         }
         String[] cluster = new String[locs.size()];
@@ -1476,4 +1495,32 @@
         return FormatUtils.getDefaultFormat();
     }
 
+    /**
+     * Add HDFS scheduler and the cluster location constraint into the scheduler
+     * 
+     * @param properties
+     *            the original dataset properties
+     * @return a new map containing the original dataset properties and the scheduler/locations
+     */
+    private Map<String, Object> wrapProperties(Map<String, String> properties) {
+        Map<String, Object> wrappedProperties = new HashMap<String, Object>();
+        wrappedProperties.putAll(properties);
+        wrappedProperties.put(HDFSAdapterFactory.SCHEDULER, hdfsScheduler);
+        wrappedProperties.put(HDFSAdapterFactory.CLUSTER_LOCATIONS, getClusterLocations());
+        return wrappedProperties;
+    }
+
+    /**
+     * Adapt the original properties to a string-object map
+     * 
+     * @param properties
+     *            the original properties
+     * @return the new stirng-object map
+     */
+    private Map<String, Object> wrapPropertiesEmpty(Map<String, String> properties) {
+        Map<String, Object> wrappedProperties = new HashMap<String, Object>();
+        wrappedProperties.putAll(properties);
+        return wrappedProperties;
+    }
+
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index 7cfa6d0..6c55f12 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -29,6 +29,7 @@
 import edu.uci.ics.asterix.builders.OrderedListBuilder;
 import edu.uci.ics.asterix.builders.RecordBuilder;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataNode;
@@ -49,7 +50,6 @@
 import edu.uci.ics.asterix.om.types.AbstractCollectionType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
index 60f2e6d..406b835 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
@@ -30,7 +30,7 @@
 
     public static void addMetadataBuiltinFunctions() {
 
-        AsterixBuiltinFunctions.add(AsterixBuiltinFunctions.DATASET, new IResultTypeComputer() {
+        AsterixBuiltinFunctions.addFunction(AsterixBuiltinFunctions.DATASET, new IResultTypeComputer() {
 
             @Override
             public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
@@ -73,7 +73,7 @@
             }
         });
 
-        AsterixBuiltinFunctions.add(AsterixBuiltinFunctions.FEED_INGEST, new IResultTypeComputer() {
+        AsterixBuiltinFunctions.addPrivateFunction(AsterixBuiltinFunctions.FEED_INGEST, new IResultTypeComputer() {
 
             @Override
             public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
index e554643..68ff549 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
@@ -19,11 +19,11 @@
 import java.io.DataInput;
 import java.io.DataInputStream;
 
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
 import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
index 2e4dbe4..62e7a28 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
@@ -20,12 +20,12 @@
 import java.io.DataInputStream;
 import java.rmi.RemoteException;
 
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataNode;
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
 import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
index 7ae334e..9593783 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
@@ -17,10 +17,10 @@
 
 import java.io.IOException;
 
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.api.IMetadataEntityTupleTranslator;
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
index d4109cd..81da41b 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
@@ -19,11 +19,11 @@
 import java.io.DataInput;
 import java.io.DataInputStream;
 
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
 import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
index 03bc963..fbf8cc8 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
@@ -18,9 +18,9 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
diff --git a/asterix-om/.gitignore b/asterix-om/.gitignore
new file mode 100644
index 0000000..ea8c4bf
--- /dev/null
+++ b/asterix-om/.gitignore
@@ -0,0 +1 @@
+/target
diff --git a/asterix-om/pom.xml b/asterix-om/pom.xml
index f7ef8d3..ac1aa7f 100644
--- a/asterix-om/pom.xml
+++ b/asterix-om/pom.xml
@@ -30,6 +30,12 @@
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-transactions</artifactId>
+			<version>0.0.6-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
 		</dependency>
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
index f795b17..a6ac8f2 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
@@ -51,7 +51,7 @@
 
             @Override
             public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-
+            	
                 if (b1[s1] == ATypeTag.NULL.serialize()) {
                     if (b2[s2] == ATypeTag.NULL.serialize())
                         return 0;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
new file mode 100644
index 0000000..6e80fe6
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
@@ -0,0 +1,165 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
+
+import edu.uci.ics.asterix.formats.nontagged.UTF8StringLowercasePointable;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class ListItemBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final ListItemBinaryComparatorFactory INSTANCE = new ListItemBinaryComparatorFactory();
+
+    private ListItemBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return createBinaryComparator(ATypeTag.NULL, ATypeTag.NULL, false);
+    }
+
+    public IBinaryComparator createBinaryComparator(final ATypeTag firstItemTypeTag, final ATypeTag secondItemTypeTag,
+            final boolean ignoreCase) {
+        return new IBinaryComparator() {
+            final IBinaryComparator ascBoolComp = BooleanBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+            final IBinaryComparator ascIntComp = new PointableBinaryComparatorFactory(IntegerPointable.FACTORY)
+                    .createBinaryComparator();
+            final IBinaryComparator ascLongComp = LongBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+            final IBinaryComparator ascStrComp = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY)
+                    .createBinaryComparator();
+            final IBinaryComparator ascLowerCaseStrComp = new PointableBinaryComparatorFactory(
+                    UTF8StringLowercasePointable.FACTORY).createBinaryComparator();
+            final IBinaryComparator ascFloatComp = new PointableBinaryComparatorFactory(FloatPointable.FACTORY)
+                    .createBinaryComparator();
+            final IBinaryComparator ascDoubleComp = new PointableBinaryComparatorFactory(DoublePointable.FACTORY)
+                    .createBinaryComparator();
+            final IBinaryComparator ascRectangleComp = ARectanglePartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator ascCircleComp = ACirclePartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator ascDurationComp = ADurationPartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator ascIntervalComp = AIntervalPartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator ascLineComp = ALinePartialBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+            final IBinaryComparator ascPointComp = APointPartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator ascPoint3DComp = APoint3DPartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator ascPolygonComp = APolygonPartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator rawComp = RawBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+
+                if (b1[s1] == ATypeTag.NULL.serialize()) {
+                    if (b2[s2] == ATypeTag.NULL.serialize())
+                        return 0;
+                    else
+                        return -1;
+                } else {
+                    if (b2[s2] == ATypeTag.NULL.serialize())
+                        return 1;
+                }
+
+                ATypeTag tag1 = firstItemTypeTag;
+                int skip1 = 0;
+                if (firstItemTypeTag == ATypeTag.ANY) {
+                    tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b1[s1]);
+                    skip1 = 1;
+                }
+
+                ATypeTag tag2 = secondItemTypeTag;
+                int skip2 = 0;
+                if (secondItemTypeTag == ATypeTag.ANY) {
+                    tag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b2[s2]);
+                    skip2 = 1;
+                }
+
+                if (tag1 != tag2) {
+                    return rawComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                }
+
+                switch (tag1) {
+                    case BOOLEAN: {
+                        return ascBoolComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case TIME:
+                    case DATE:
+                    case YEARMONTHDURATION:
+                    case INT32: {
+                        return ascIntComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case DATETIME:
+                    case DAYTIMEDURATION:
+                    case INT64: {
+                        return ascLongComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case FLOAT: {
+                        return ascFloatComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case DOUBLE: {
+                        return ascDoubleComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case STRING: {
+                        if (ignoreCase) {
+                            return ascLowerCaseStrComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                        } else {
+                            return ascStrComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                        }
+                    }
+                    case RECTANGLE: {
+                        return ascRectangleComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case CIRCLE: {
+                        return ascCircleComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case POINT: {
+                        return ascPointComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case POINT3D: {
+                        return ascPoint3DComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case LINE: {
+                        return ascLineComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case POLYGON: {
+                        return ascPolygonComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case DURATION: {
+                        return ascDurationComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    case INTERVAL: {
+                        return ascIntervalComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                    default: {
+                        return rawComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
+                }
+            }
+        };
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/AObjectBinaryHashFunctionFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/AObjectBinaryHashFunctionFactory.java
index 495b41f..0946f1b 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/AObjectBinaryHashFunctionFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/AObjectBinaryHashFunctionFactory.java
@@ -1,14 +1,8 @@
 package edu.uci.ics.asterix.dataflow.data.nontagged.hash;
 
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.EnumDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.data.std.accessors.MurmurHash3BinaryHashFunctionFamily;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
-import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class AObjectBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
 
@@ -22,56 +16,12 @@
     @Override
     public IBinaryHashFunction createBinaryHashFunction() {
         return new IBinaryHashFunction() {
-
-            private IBinaryHashFunction boolHash = BooleanBinaryHashFunctionFactory.INSTANCE.createBinaryHashFunction();
-            private IBinaryHashFunction intHash = new PointableBinaryHashFunctionFactory(IntegerPointable.FACTORY)
-                    .createBinaryHashFunction();
-            private IBinaryHashFunction longHash = LongBinaryHashFunctionFactory.INSTANCE.createBinaryHashFunction();
-            private IBinaryHashFunction floatHash = new PointableBinaryHashFunctionFactory(FloatPointable.FACTORY)
-                    .createBinaryHashFunction();
-            private IBinaryHashFunction stringHash = new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY)
-                    .createBinaryHashFunction();
-
-            private IBinaryHashFunction doubleHash = DoubleBinaryHashFunctionFactory.INSTANCE
-                    .createBinaryHashFunction();
-
             private IBinaryHashFunction genericBinaryHash = MurmurHash3BinaryHashFunctionFamily.INSTANCE
                     .createBinaryHashFunction(0);
 
             @Override
             public int hash(byte[] bytes, int offset, int length) {
-                ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
-                switch (tag) {
-                    case BOOLEAN: {
-                        return boolHash.hash(bytes, offset + 1, length - 1);
-                    }
-                    case TIME:
-                    case DATE:
-                    case YEARMONTHDURATION:
-                    case INT32: {
-                        return intHash.hash(bytes, offset + 1, length - 1);
-                    }
-                    case DATETIME:
-                    case DAYTIMEDURATION:
-                    case INT64: {
-                        return longHash.hash(bytes, offset + 1, length - 1);
-                    }
-                    case FLOAT: {
-                        return floatHash.hash(bytes, offset + 1, length - 1);
-                    }
-                    case DOUBLE: {
-                        return doubleHash.hash(bytes, offset + 1, length - 1);
-                    }
-                    case STRING: {
-                        return stringHash.hash(bytes, offset + 1, length - 1);
-                    }
-                    case NULL: {
-                        return 0;
-                    }
-                    default: {
-                        return genericBinaryHash.hash(bytes, offset + 1, length - 1);
-                    }
-                }
+                return genericBinaryHash.hash(bytes, offset, length);
             }
         };
     }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/ListItemBinaryHashFunctionFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/ListItemBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..0fab7de
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/ListItemBinaryHashFunctionFactory.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.dataflow.data.nontagged.hash;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.formats.nontagged.UTF8StringLowercasePointable;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.MurmurHash3BinaryHashFunctionFamily;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+
+/**
+ * This hash function factory is introduced to be able to hash heterogeneous list items.
+ * The item type tag is also included in the hash computation to distinguish the different
+ * types with the same raw bytes.
+ */
+public class ListItemBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final ListItemBinaryHashFunctionFactory INSTANCE = new ListItemBinaryHashFunctionFactory();
+
+    private ListItemBinaryHashFunctionFactory() {
+    }
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction() {
+        return createBinaryHashFunction(ATypeTag.ANY, false);
+    }
+
+    public IBinaryHashFunction createBinaryHashFunction(final ATypeTag itemTypeTag, final boolean ignoreCase) {
+        return new IBinaryHashFunction() {
+
+            private IBinaryHashFunction lowerCaseStringHash = new PointableBinaryHashFunctionFactory(
+                    UTF8StringLowercasePointable.FACTORY).createBinaryHashFunction();
+            private IBinaryHashFunction genericBinaryHash = MurmurHash3BinaryHashFunctionFamily.INSTANCE
+                    .createBinaryHashFunction(0);
+            private GrowableArray taggedBytes = new GrowableArray();
+
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                ATypeTag tag = itemTypeTag;
+                int skip = 0;
+                if (itemTypeTag == ATypeTag.ANY) {
+                    tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
+                    skip = 1;
+                }
+                switch (tag) {
+                    case STRING: {
+                        if (ignoreCase) {
+                            return lowerCaseStringHash.hash(bytes, offset + skip, length - skip);
+                        }
+                    }
+                    default: {
+                        if (itemTypeTag != ATypeTag.ANY) {
+                            // add the itemTypeTag in front of the data
+                            try {
+                                resetTaggedBytes(bytes, offset, length);
+                                return genericBinaryHash.hash(taggedBytes.getByteArray(), 0, length + 1);
+                            } catch (IOException e) {
+                                throw new RuntimeException(e);
+                            }
+                        } else {
+                            return genericBinaryHash.hash(bytes, offset, length);
+                        }
+                    }
+                }
+            }
+
+            private void resetTaggedBytes(byte[] data, int offset, int length) throws IOException {
+                taggedBytes.reset();
+                taggedBytes.getDataOutput().writeByte(itemTypeTag.serialize());
+                taggedBytes.getDataOutput().write(data, offset, length);
+            }
+        };
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADatePrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADatePrinter.java
index 327faf1..b7df6ff 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADatePrinter.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADatePrinter.java
@@ -1,9 +1,11 @@
 package edu.uci.ics.asterix.dataflow.data.nontagged.printers;
 
+import java.io.IOException;
 import java.io.PrintStream;
 
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
 import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem.Fields;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.data.IPrinter;
 
@@ -22,22 +24,22 @@
     public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
         long chrononTime = AInt32SerializerDeserializer.getInt(b, s + 1) * CHRONON_OF_DAY;
 
-        int year = gCalInstance.getYear(chrononTime);
-        int month = gCalInstance.getMonthOfYear(chrononTime, year);
-
         ps.print("date(\"");
-        ps.append(String.format(year < 0 ? "%05d" : "%04d", year)).append("-").append(String.format("%02d", month))
-                .append("-").append(String.format("%02d", gCalInstance.getDayOfMonthYear(chrononTime, year, month)));
+        try {
+            gCalInstance.getExtendStringRepUntilField(chrononTime, 0, ps, Fields.YEAR, Fields.DAY, false);
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
         ps.print("\")");
     }
 
     public void printString(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
         long chrononTime = AInt32SerializerDeserializer.getInt(b, s + 1) * CHRONON_OF_DAY;
 
-        int year = gCalInstance.getYear(chrononTime);
-        int month = gCalInstance.getMonthOfYear(chrononTime, year);
-
-        ps.append(String.format(year < 0 ? "%05d" : "%04d", year)).append("-").append(String.format("%02d", month))
-                .append("-").append(String.format("%02d", gCalInstance.getDayOfMonthYear(chrononTime, year, month)));
+        try {
+            gCalInstance.getExtendStringRepUntilField(chrononTime, 0, ps, Fields.YEAR, Fields.DAY, false);
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
     }
 }
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADateTimePrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADateTimePrinter.java
index 7cf1478..a412cb1 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADateTimePrinter.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADateTimePrinter.java
@@ -1,9 +1,11 @@
 package edu.uci.ics.asterix.dataflow.data.nontagged.printers;
 
+import java.io.IOException;
 import java.io.PrintStream;
 
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
 import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem.Fields;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.data.IPrinter;
 
@@ -21,30 +23,25 @@
     public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
         long chrononTime = AInt64SerializerDeserializer.getLong(b, s + 1);
 
-        int year = gCalInstance.getYear(chrononTime);
-        int month = gCalInstance.getMonthOfYear(chrononTime, year);
-
         ps.print("datetime(\"");
-        ps.append(String.format(year < 0 ? "%05d" : "%04d", year)).append("-").append(String.format("%02d", month))
-                .append("-").append(String.format("%02d", gCalInstance.getDayOfMonthYear(chrononTime, year, month)))
-                .append("T").append(String.format("%02d", gCalInstance.getHourOfDay(chrononTime))).append(":")
-                .append(String.format("%02d", gCalInstance.getMinOfHour(chrononTime))).append(":")
-                .append(String.format("%02d", gCalInstance.getSecOfMin(chrononTime))).append(".")
-                .append(String.format("%03d", gCalInstance.getMillisOfSec(chrononTime))).append("Z");
+
+        try {
+            gCalInstance.getExtendStringRepUntilField(chrononTime, 0, ps, Fields.YEAR, Fields.MILLISECOND, true);
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
+
         ps.print("\")");
     }
 
     public void printString(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
         long chrononTime = AInt64SerializerDeserializer.getLong(b, s + 1);
 
-        int year = gCalInstance.getYear(chrononTime);
-        int month = gCalInstance.getMonthOfYear(chrononTime, year);
-
-        ps.append(String.format(year < 0 ? "%05d" : "%04d", year)).append("-").append(String.format("%02d", month))
-                .append("-").append(String.format("%02d", gCalInstance.getDayOfMonthYear(chrononTime, year, month)))
-                .append("T").append(String.format("%02d", gCalInstance.getHourOfDay(chrononTime))).append(":")
-                .append(String.format("%02d", gCalInstance.getMinOfHour(chrononTime))).append(":")
-                .append(String.format("%02d", gCalInstance.getSecOfMin(chrononTime))).append(".")
-                .append(String.format("%03d", gCalInstance.getMillisOfSec(chrononTime))).append("Z");
+        try {
+            gCalInstance.getExtendStringRepUntilField(chrononTime, 0, ps, Fields.YEAR, Fields.MILLISECOND, true);
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
+        
     }
 }
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ATimePrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ATimePrinter.java
index aba6a4b..6550906 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ATimePrinter.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ATimePrinter.java
@@ -1,9 +1,11 @@
 package edu.uci.ics.asterix.dataflow.data.nontagged.printers;
 
+import java.io.IOException;
 import java.io.PrintStream;
 
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
 import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem.Fields;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.data.IPrinter;
 
@@ -22,21 +24,24 @@
         int time = AInt32SerializerDeserializer.getInt(b, s + 1);
 
         ps.print("time(\"");
-        ps.append(String.format("%02d", gCalInstance.getHourOfDay(time))).append(":")
-                .append(String.format("%02d", gCalInstance.getMinOfHour(time))).append(":")
-                .append(String.format("%02d", gCalInstance.getSecOfMin(time))).append(".")
-                .append(String.format("%03d", gCalInstance.getMillisOfSec(time))).append("Z");
+
+        try {
+            gCalInstance.getExtendStringRepUntilField(time, 0, ps, Fields.HOUR, Fields.MILLISECOND, true);
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
 
         ps.print("\")");
     }
-    
+
     public void printString(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
         int time = AInt32SerializerDeserializer.getInt(b, s + 1);
 
-        ps.append(String.format("%02d", gCalInstance.getHourOfDay(time))).append(":")
-                .append(String.format("%02d", gCalInstance.getMinOfHour(time))).append(":")
-                .append(String.format("%02d", gCalInstance.getSecOfMin(time))).append(".")
-                .append(String.format("%03d", gCalInstance.getMillisOfSec(time))).append("Z");
+        try {
+            gCalInstance.getExtendStringRepUntilField(time, 0, ps, Fields.HOUR, Fields.MILLISECOND, true);
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
     }
 
 }
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java
index 11f65dc..51e0413 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.algebricks.data.ITypeTraitProvider;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
 
 public interface IDataFormat {
@@ -68,4 +69,6 @@
     public INormalizedKeyComputerFactoryProvider getNormalizedKeyComputerFactoryProvider();
 
     public IBinaryHashFunctionFamilyProvider getBinaryHashFunctionFamilyProvider();
+    
+    public IPredicateEvaluatorFactoryProvider getPredicateEvaluatorFactoryProvider();
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java
index 7bfdb050..76dd07d 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java
@@ -2,11 +2,6 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.asterix.dataflow.data.nontagged.hash.AObjectBinaryHashFunctionFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.hash.BooleanBinaryHashFunctionFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.hash.DoubleBinaryHashFunctionFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.hash.LongBinaryHashFunctionFactory;
-import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
@@ -16,7 +11,6 @@
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.primitive.RawUTF8StringPointable;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class AqlBinaryHashFunctionFactoryProvider implements IBinaryHashFunctionFactoryProvider, Serializable {
 
@@ -39,81 +33,6 @@
 
     @Override
     public IBinaryHashFunctionFactory getBinaryHashFunctionFactory(Object type) {
-        if (type == null) {
-            return AObjectBinaryHashFunctionFactory.INSTANCE;
-        }
-        IAType aqlType = (IAType) type;
-        switch (aqlType.getTypeTag()) {
-            case ANY:
-            case UNION: { // we could do smth better for nullable fields
-                return AObjectBinaryHashFunctionFactory.INSTANCE;
-            }
-            case NULL: {
-                return new IBinaryHashFunctionFactory() {
-
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public IBinaryHashFunction createBinaryHashFunction() {
-                        return new IBinaryHashFunction() {
-
-                            @Override
-                            public int hash(byte[] bytes, int offset, int length) {
-                                return 0;
-                            }
-                        };
-                    }
-                };
-            }
-            case BOOLEAN: {
-                return addOffset(BooleanBinaryHashFunctionFactory.INSTANCE);
-            }
-            case DATE:
-            case TIME:
-            case YEARMONTHDURATION:
-            case INT32: {
-                return addOffset(new PointableBinaryHashFunctionFactory(IntegerPointable.FACTORY));
-            }
-            case DAYTIMEDURATION:
-            case DATETIME:
-            case INT64: {
-                return addOffset(LongBinaryHashFunctionFactory.INSTANCE);
-            }
-            case FLOAT: {
-                return addOffset(new PointableBinaryHashFunctionFactory(FloatPointable.FACTORY));
-            }
-            case DOUBLE: {
-                return addOffset(DoubleBinaryHashFunctionFactory.INSTANCE);
-            }
-            case STRING: {
-                return addOffset(new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY));
-            }
-            default: {
-                return addOffsetForGenericBinaryHash();
-            }
-        }
-    }
-
-    private IBinaryHashFunctionFactory addOffset(final IBinaryHashFunctionFactory inst) {
-        return new IBinaryHashFunctionFactory() {
-
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public IBinaryHashFunction createBinaryHashFunction() {
-                final IBinaryHashFunction bhf = inst.createBinaryHashFunction();
-                return new IBinaryHashFunction() {
-
-                    @Override
-                    public int hash(byte[] bytes, int offset, int length) {
-                        return bhf.hash(bytes, offset + 1, length - 1);
-                    }
-                };
-            }
-        };
-    }
-
-    private IBinaryHashFunctionFactory addOffsetForGenericBinaryHash() {
         return new IBinaryHashFunctionFactory() {
 
             private static final long serialVersionUID = 1L;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlPredicateEvaluatorFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlPredicateEvaluatorFactoryProvider.java
new file mode 100644
index 0000000..f4fa74e
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlPredicateEvaluatorFactoryProvider.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.formats.nontagged;
+
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
+
+/*
+Provides PredicateEvaluator for equi-join cases to properly take care of NULL fields, being compared with each other.
+If any of the join keys, from either side, is NULL, record should not pass equi-join condition.
+*/
+public class AqlPredicateEvaluatorFactoryProvider implements IPredicateEvaluatorFactoryProvider{
+	
+	private static final long serialVersionUID = 1L;
+	public static final AqlPredicateEvaluatorFactoryProvider INSTANCE = new AqlPredicateEvaluatorFactoryProvider();
+	
+	@Override
+	public IPredicateEvaluatorFactory getPredicateEvaluatorFactory(final int[] keys0, final int[] keys1) {
+		
+		return new IPredicateEvaluatorFactory() {
+			private static final long serialVersionUID = 1L;
+			@Override
+			public IPredicateEvaluator createPredicateEvaluator() {
+				return new IPredicateEvaluator() {
+					
+					@Override
+					public boolean evaluate(IFrameTupleAccessor fta0, int tupId0,
+							IFrameTupleAccessor fta1, int tupId1) {
+						
+						int tStart0 = fta0.getTupleStartOffset(tupId0);
+				        int fStartOffset0 = fta0.getFieldSlotsLength() + tStart0;
+						
+						for(int k0 : keys0){
+							int fieldStartIx = fta0.getFieldStartOffset(tupId0, k0);
+							ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(fta0.getBuffer().array()[fieldStartIx + fStartOffset0]);
+							if(typeTag == ATypeTag.NULL){
+								return false;
+							}
+						}
+						
+						int tStart1 = fta1.getTupleStartOffset(tupId1);
+				        int fStartOffset1 = fta1.getFieldSlotsLength() + tStart1;
+				        
+						for(int k1 : keys1){
+							int fieldStartIx = fta1.getFieldStartOffset(tupId1, k1);
+							ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(fta1.getBuffer().array()[fieldStartIx + fStartOffset1]);
+							if(typeTag == ATypeTag.NULL){
+								return false;
+							}
+						}
+						
+						return true;	//none of the fields (from both sides) is NULL
+					}
+				};
+			}
+		};
+	}
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADate.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADate.java
index 0ef170e..28008b2 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADate.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADate.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.asterix.om.base;
 
+import java.io.IOException;
+
 import org.json.JSONException;
 import org.json.JSONObject;
 
@@ -75,9 +77,12 @@
     public String toString() {
         StringBuilder sbder = new StringBuilder();
         sbder.append("ADate: { ");
-        GregorianCalendarSystem.getInstance().getExtendStringRepWithTimezoneUntilField(
-                chrononTimeInDay * CHRONON_OF_DAY, 0, sbder, GregorianCalendarSystem.Fields.YEAR,
-                GregorianCalendarSystem.Fields.DAY);
+        try {
+            GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(chrononTimeInDay * CHRONON_OF_DAY, 0,
+                    sbder, GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.DAY, false);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
         sbder.append(" }");
         return sbder.toString();
     }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADateTime.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADateTime.java
index 081fa63..10d49aa 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADateTime.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADateTime.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.asterix.om.base;
 
+import java.io.IOException;
+
 import org.json.JSONException;
 import org.json.JSONObject;
 
@@ -106,8 +108,12 @@
     public String toString() {
         StringBuilder sbder = new StringBuilder();
         sbder.append("ADateTime: { ");
-        GregorianCalendarSystem.getInstance().getExtendStringRepWithTimezoneUntilField(chrononTime, 0, sbder,
-                GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND);
+        try {
+            GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(chrononTime, 0, sbder,
+                    GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
         sbder.append(" }");
         return sbder.toString();
     }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInterval.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInterval.java
index f1352f1..e3c5213 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInterval.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInterval.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.asterix.om.base;
 
+import java.io.IOException;
+
 import org.json.JSONException;
 import org.json.JSONObject;
 
@@ -100,33 +102,38 @@
     public String toString() {
         StringBuilder sbder = new StringBuilder();
         sbder.append("AInterval: { ");
-        if (typetag == ATypeTag.DATE.serialize()) {
-            sbder.append("ADate: { ");
-            GregorianCalendarSystem.getInstance().getExtendStringRepWithTimezoneUntilField(
-                    intervalStart * ADate.CHRONON_OF_DAY, 0, sbder, GregorianCalendarSystem.Fields.YEAR,
-                    GregorianCalendarSystem.Fields.DAY);
-            sbder.append(" }, ADate: {");
-            GregorianCalendarSystem.getInstance().getExtendStringRepWithTimezoneUntilField(
-                    intervalEnd * ADate.CHRONON_OF_DAY, 0, sbder, GregorianCalendarSystem.Fields.YEAR,
-                    GregorianCalendarSystem.Fields.DAY);
-            sbder.append(" }");
-        } else if (typetag == ATypeTag.TIME.serialize()) {
-            sbder.append("ATime: { ");
-            GregorianCalendarSystem.getInstance().getExtendStringRepWithTimezoneUntilField(intervalStart, 0, sbder,
-                    GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND);
-            sbder.append(" }, ATime: { ");
+        try {
+            if (typetag == ATypeTag.DATE.serialize()) {
+                sbder.append("ADate: { ");
 
-            GregorianCalendarSystem.getInstance().getExtendStringRepWithTimezoneUntilField(intervalEnd, 0, sbder,
-                    GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND);
-            sbder.append(" }");
-        } else if (typetag == ATypeTag.DATETIME.serialize()) {
-            sbder.append("ADateTime: { ");
-            GregorianCalendarSystem.getInstance().getExtendStringRepWithTimezoneUntilField(intervalStart, 0, sbder,
-                    GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND);
-            sbder.append(" }, ADateTime: { ");
-            GregorianCalendarSystem.getInstance().getExtendStringRepWithTimezoneUntilField(intervalEnd, 0, sbder,
-                    GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND);
-            sbder.append(" }");
+                GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(
+                        intervalStart * ADate.CHRONON_OF_DAY, 0, sbder, GregorianCalendarSystem.Fields.YEAR,
+                        GregorianCalendarSystem.Fields.DAY, false);
+
+                sbder.append(" }, ADate: {");
+                GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalEnd * ADate.CHRONON_OF_DAY,
+                        0, sbder, GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.DAY, false);
+                sbder.append(" }");
+            } else if (typetag == ATypeTag.TIME.serialize()) {
+                sbder.append("ATime: { ");
+                GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalStart, 0, sbder,
+                        GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+                sbder.append(" }, ATime: { ");
+
+                GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalEnd, 0, sbder,
+                        GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+                sbder.append(" }");
+            } else if (typetag == ATypeTag.DATETIME.serialize()) {
+                sbder.append("ADateTime: { ");
+                GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalStart, 0, sbder,
+                        GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+                sbder.append(" }, ADateTime: { ");
+                GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalEnd, 0, sbder,
+                        GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+                sbder.append(" }");
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
         }
         sbder.append(" }");
         return sbder.toString();
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ATime.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ATime.java
index b97ed51..edbf007 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ATime.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ATime.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.asterix.om.base;
 
+import java.io.IOException;
+
 import org.json.JSONException;
 import org.json.JSONObject;
 
@@ -90,8 +92,12 @@
     public String toString() {
         StringBuilder sbder = new StringBuilder();
         sbder.append("ATime: { ");
-        GregorianCalendarSystem.getInstance().getExtendStringRepWithTimezoneUntilField(chrononTime, 0, sbder,
-                GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND);
+        try {
+            GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(chrononTime, 0, sbder,
+                    GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
         sbder.append(" }");
         return sbder.toString();
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ADurationParserFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ADurationParserFactory.java
index 84e2386..a651716 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ADurationParserFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ADurationParserFactory.java
@@ -248,18 +248,30 @@
             throw new HyracksDataException(durationErrorMessage + ": no time fields after time separator.");
         }
 
+        int totalMonths = sign * (year * 12 + month);
+        long totalMilliseconds = sign
+                * (day * GregorianCalendarSystem.CHRONON_OF_DAY + hour * GregorianCalendarSystem.CHRONON_OF_HOUR
+                        + minute * GregorianCalendarSystem.CHRONON_OF_MINUTE + second
+                        * GregorianCalendarSystem.CHRONON_OF_SECOND + millisecond);
+
+        if (sign > 0) {
+            if (totalMonths < 0) {
+                throw new HyracksDataException(durationErrorMessage
+                        + ": total number of months is beyond its max value (-2147483647 to 2147483647).");
+            }
+            if (totalMilliseconds < 0) {
+                throw new HyracksDataException(
+                        durationErrorMessage
+                                + ": total number of milliseconds is beyond its max value (-9223372036854775808 to 9223372036854775807).");
+            }
+        }
+
         if (mutableObject instanceof AMutableDuration) {
-            ((AMutableDuration) mutableObject)
-                    .setValue(sign * (year * 12 + month),
-                            sign
-                                    * (day * 24 * 3600 * 1000L + 3600 * 1000L * hour + 60 * minute * 1000L + second
-                                            * 1000L + millisecond));
+            ((AMutableDuration) mutableObject).setValue(totalMonths, totalMilliseconds);
         } else if (mutableObject instanceof AMutableYearMonthDuration) {
-            ((AMutableYearMonthDuration) mutableObject).setMonths(sign * (year * 12 + month));
+            ((AMutableYearMonthDuration) mutableObject).setMonths(totalMonths);
         } else if (mutableObject instanceof AMutableDayTimeDuration) {
-            ((AMutableDayTimeDuration) mutableObject)
-                    .setMilliseconds(sign
-                            * (day * 24 * 3600 * 1000L + 3600 * 1000L * hour + 60 * minute * 1000L + second * 1000L + millisecond));
+            ((AMutableDayTimeDuration) mutableObject).setMilliseconds(totalMilliseconds);
         }
     }
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java
index f74af9b..8c21bad 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java
@@ -30,6 +30,8 @@
  */
 package edu.uci.ics.asterix.om.base.temporal;
 
+import java.io.IOException;
+
 /**
  * A simple implementation of the Gregorian calendar system.
  * <p/>
@@ -53,9 +55,9 @@
 
     public static final int[] DAYS_SINCE_MONTH_BEGIN_ORDI = { 0, 31, 59, 90, 120, 151, 181, 212, 243, 273, 304, 334 };
 
-    public static final int CHRONON_OF_SECOND = 1000;
-    public static final int CHRONON_OF_MINUTE = 60 * CHRONON_OF_SECOND;
-    public static final int CHRONON_OF_HOUR = 60 * CHRONON_OF_MINUTE;
+    public static final long CHRONON_OF_SECOND = 1000;
+    public static final long CHRONON_OF_MINUTE = 60 * CHRONON_OF_SECOND;
+    public static final long CHRONON_OF_HOUR = 60 * CHRONON_OF_MINUTE;
     public static final long CHRONON_OF_DAY = 24 * CHRONON_OF_HOUR;
     public static final int MONTHS_IN_A_YEAR = 12;
 
@@ -234,9 +236,9 @@
      */
     public int getChronon(int hour, int min, int sec, int millis, int timezone) {
         // Added milliseconds for all fields but month and day
-        int chrononTime = (hour - timezone / 4) * CHRONON_OF_HOUR + (min - (timezone % 4) * 15) * CHRONON_OF_MINUTE
+        long chrononTime = (hour - timezone / 4) * CHRONON_OF_HOUR + (min - (timezone % 4) * 15) * CHRONON_OF_MINUTE
                 + sec * CHRONON_OF_SECOND + millis;
-        return chrononTime;
+        return (int)chrononTime;
     }
 
     public long adjustChrononByTimezone(long chronon, int timezone) {
@@ -268,8 +270,8 @@
      * @param sbder
      * @param untilField
      */
-    public void getExtendStringRepWithTimezoneUntilField(long chrononTime, int timezone, StringBuilder sbder,
-            Fields startField, Fields untilField) {
+    public void getExtendStringRepUntilField(long chrononTime, int timezone, Appendable sbder, Fields startField,
+            Fields untilField, boolean withTimezone) throws IOException {
 
         int year = getYear(chrononTime);
         int month = getMonthOfYear(chrononTime, year);
@@ -323,16 +325,18 @@
                 break;
         }
 
-        if (timezone == 0) {
-            sbder.append("Z");
-        } else {
-            short tzMin = (short) ((timezone % 4) * 15);
-            if (tzMin < 0) {
-                tzMin = (short) (-1 * tzMin);
+        if (withTimezone) {
+            if (timezone == 0) {
+                sbder.append("Z");
+            } else {
+                short tzMin = (short) ((timezone % 4) * 15);
+                if (tzMin < 0) {
+                    tzMin = (short) (-1 * tzMin);
+                }
+                short tzHr = (short) (timezone / 4);
+                sbder.append((tzHr >= 0 ? "+" : "-")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
+                        .append(":").append(String.format("%02d", tzMin));
             }
-            short tzHr = (short) (timezone / 4);
-            sbder.append((tzHr >= 0 ? "+" : "-")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr))).append(":")
-                    .append(String.format("%02d", tzMin));
         }
     }
 
@@ -343,8 +347,8 @@
      * @param timezone
      * @param sbder
      */
-    public void getBasicStringRepWithTimezoneUntilField(long chrononTime, int timezone, StringBuilder sbder,
-            Fields startField, Fields untilField) {
+    public void getBasicStringRepUntilField(long chrononTime, int timezone, Appendable sbder, Fields startField,
+            Fields untilField, boolean withTimezone) throws IOException {
         int year = getYear(chrononTime);
         int month = getMonthOfYear(chrononTime, year);
 
@@ -382,16 +386,18 @@
                 break;
         }
 
-        if (timezone == 0) {
-            sbder.append("Z");
-        } else {
-            short tzMin = (short) ((timezone % 4) * 15);
-            if (tzMin < 0) {
-                tzMin = (short) (-1 * tzMin);
+        if (withTimezone) {
+            if (timezone == 0) {
+                sbder.append("Z");
+            } else {
+                short tzMin = (short) ((timezone % 4) * 15);
+                if (tzMin < 0) {
+                    tzMin = (short) (-1 * tzMin);
+                }
+                short tzHr = (short) (timezone / 4);
+                sbder.append((tzHr >= 0 ? "+" : "-")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
+                        .append(String.format("%02d", tzMin));
             }
-            short tzHr = (short) (timezone / 4);
-            sbder.append((tzHr >= 0 ? "+" : "-")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
-                    .append(String.format("%02d", tzMin));
         }
     }
 
@@ -480,8 +486,7 @@
      * Get the year for the given chronon time.
      * <p/>
      * This code is directly from the Joda library BadicChronology.java.<br/>
-     * The original authers are Stephen Colebourne, Brain S O'Neill and Guy Allard, and modified by JArod Wen on May
-     * 7th, 2012.
+     * The original authers are Stephen Colebourne, Brain S O'Neill and Guy Allard, and modified by JArod Wen on May 7th, 2012.
      * 
      * @param chrononTime
      * @return
@@ -528,8 +533,7 @@
      * Get the month of the year for the given chronon time and the year.
      * <p/>
      * This code is directly from the Joda library BasicGJChronology.java.<br/>
-     * The original authers are Stephen Colebourne, Brain S O'Neill and Guy Allard, and modified by JArod Wen on May
-     * 7th, 2012 and commented by Theodoros Ioannou on July 2012.
+     * The original authers are Stephen Colebourne, Brain S O'Neill and Guy Allard, and modified by JArod Wen on May 7th, 2012 and commented by Theodoros Ioannou on July 2012.
      * <p/>
      * 
      * @param millis
@@ -593,8 +597,7 @@
      * Get the day of the given month and year for the input chronon time.
      * <p/>
      * This function is directly from Joda Library BasicChronology.java.<br/>
-     * The original authers are Stephen Colebourne, Brain S O'Neill and Guy Allard, and modified by JArod Wen on May
-     * 7th, 2012.
+     * The original authers are Stephen Colebourne, Brain S O'Neill and Guy Allard, and modified by JArod Wen on May 7th, 2012.
      * <p/>
      * 
      * @param millis
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
index 2ae523e..8e40200 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -8,6 +8,8 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ABooleanTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ACircleTypeComputer;
@@ -30,6 +32,7 @@
 import edu.uci.ics.asterix.om.typecomputer.impl.CastListResultTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.CastRecordResultTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ClosedRecordConstructorResultType;
+import edu.uci.ics.asterix.om.typecomputer.impl.CollectionToSequenceTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ConcatNonNullTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
 import edu.uci.ics.asterix.om.typecomputer.impl.InjectFailureTypeComputer;
@@ -104,18 +107,12 @@
         SI
     }
 
-    /*
-     * A function is categorized as public or private depending upon whether it can be used by the end-user in AQL queries.
-     */
-    public enum FunctionNamespace {
-        ASTERIX_PUBLIC,
-        ASTERIX_PRIVATE
-    }
-
     private static final FunctionInfoRepository registeredFunctions = new FunctionInfoRepository();
 
     // it is supposed to be an identity mapping
-    private final static Map<IFunctionInfo, IFunctionInfo> builtinFunctionsSet = new HashMap<IFunctionInfo, IFunctionInfo>();
+    private final static Map<IFunctionInfo, IFunctionInfo> builtinPublicFunctionsSet = new HashMap<IFunctionInfo, IFunctionInfo>();
+    private final static Map<IFunctionInfo, IFunctionInfo> builtinPrivateFunctionsSet = new HashMap<IFunctionInfo, IFunctionInfo>();
+
     private final static Map<IFunctionInfo, IResultTypeComputer> funTypeComputer = new HashMap<IFunctionInfo, IResultTypeComputer>();
 
     private final static Set<IFunctionInfo> builtinAggregateFunctions = new HashSet<IFunctionInfo>();
@@ -127,449 +124,430 @@
     private final static Map<IFunctionInfo, IFunctionInfo> scalarToAggregateFunctionMap = new HashMap<IFunctionInfo, IFunctionInfo>();
     private static final Map<IFunctionInfo, SpatialFilterKind> spatialFilterFunctions = new HashMap<IFunctionInfo, SpatialFilterKind>();
 
-    public final static FunctionIdentifier TYPE_OF = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "type-of", 1);
-    public final static FunctionIdentifier GET_HANDLE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "get-handle", 2);
-    public final static FunctionIdentifier GET_DATA = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "get-data", 2);
-    public final static FunctionIdentifier EMBED_TYPE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "embed-type", 1);
+    public final static FunctionIdentifier TYPE_OF = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "type-of", 1);
+    public final static FunctionIdentifier GET_HANDLE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "get-handle", 2);
+    public final static FunctionIdentifier GET_DATA = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-data",
+            2);
+    public final static FunctionIdentifier EMBED_TYPE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "embed-type", 1);
 
-    public final static FunctionIdentifier GET_ITEM = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "get-item", 2);
-    public final static FunctionIdentifier ANY_COLLECTION_MEMBER = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "any-collection-member", 1);
-    public final static FunctionIdentifier LISTIFY = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "listify", 1);
+    public final static FunctionIdentifier GET_ITEM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-item",
+            2);
+    public final static FunctionIdentifier ANY_COLLECTION_MEMBER = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "any-collection-member", 1);
+    public final static FunctionIdentifier LISTIFY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "listify", 1);
     // public final static FunctionIdentifier BAGIFY = new
     // FunctionIdentifier(ASTERIX_NS, "bagify", 1, true);
-    public final static FunctionIdentifier LEN = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(), "len",
-            1);
+    public final static FunctionIdentifier LEN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "len", 1);
 
-    public final static FunctionIdentifier CONCAT_NON_NULL = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "concat-non-null", FunctionIdentifier.VARARGS);
-    public final static FunctionIdentifier EMPTY_STREAM = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "empty-stream", 0);
-    public final static FunctionIdentifier NON_EMPTY_STREAM = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "non-empty-stream", 0);
+    public final static FunctionIdentifier CONCAT_NON_NULL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "concat-non-null", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier EMPTY_STREAM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "empty-stream", 0);
+    public final static FunctionIdentifier NON_EMPTY_STREAM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "non-empty-stream", 0);
     public final static FunctionIdentifier ORDERED_LIST_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "ordered-list-constructor", FunctionIdentifier.VARARGS);
+            FunctionConstants.ASTERIX_NS, "ordered-list-constructor", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier UNORDERED_LIST_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "unordered-list-constructor", FunctionIdentifier.VARARGS);
+            FunctionConstants.ASTERIX_NS, "unordered-list-constructor", FunctionIdentifier.VARARGS);
 
     // records
     public final static FunctionIdentifier CLOSED_RECORD_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "closed-record-constructor", FunctionIdentifier.VARARGS);
+            FunctionConstants.ASTERIX_NS, "closed-record-constructor", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier OPEN_RECORD_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "open-record-constructor", FunctionIdentifier.VARARGS);
+            FunctionConstants.ASTERIX_NS, "open-record-constructor", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier RECORD_TYPE_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "record-type-constructor", FunctionIdentifier.VARARGS);
-    public final static FunctionIdentifier FIELD_ACCESS_BY_INDEX = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "field-access-by-index", 2);
-    public final static FunctionIdentifier FIELD_ACCESS_BY_NAME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "field-access-by-name", 2);
+            FunctionConstants.ASTERIX_NS, "record-type-constructor", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier FIELD_ACCESS_BY_INDEX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "field-access-by-index", 2);
+    public final static FunctionIdentifier FIELD_ACCESS_BY_NAME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "field-access-by-name", 2);
 
-    public final static FunctionIdentifier NUMERIC_UNARY_MINUS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-unary-minus", 1);
+    public final static FunctionIdentifier NUMERIC_UNARY_MINUS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-unary-minus", 1);
 
-    public final static FunctionIdentifier NUMERIC_SUBTRACT = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-subtract", 2);
-    public final static FunctionIdentifier NUMERIC_MULTIPLY = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-multiply", 2);
-    public final static FunctionIdentifier NUMERIC_DIVIDE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-divide", 2);
-    public final static FunctionIdentifier NUMERIC_MOD = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-mod", 2);
-    public final static FunctionIdentifier NUMERIC_IDIV = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-idiv", 2);
-    public final static FunctionIdentifier CARET = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
-            "caret", 2);
+    public final static FunctionIdentifier NUMERIC_SUBTRACT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-subtract", 2);
+    public final static FunctionIdentifier NUMERIC_MULTIPLY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-multiply", 2);
+    public final static FunctionIdentifier NUMERIC_DIVIDE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-divide", 2);
+    public final static FunctionIdentifier NUMERIC_MOD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-mod", 2);
+    public final static FunctionIdentifier NUMERIC_IDIV = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-idiv", 2);
+    public final static FunctionIdentifier CARET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "caret", 2);
 
-    public final static FunctionIdentifier NUMERIC_ABS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-abs", 1);
-    public final static FunctionIdentifier NUMERIC_CEILING = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-ceiling", 1);
-    public final static FunctionIdentifier NUMERIC_FLOOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-floor", 1);
-    public final static FunctionIdentifier NUMERIC_ROUND = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-round", 1);
+    public final static FunctionIdentifier NUMERIC_ABS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-abs", 1);
+    public final static FunctionIdentifier NUMERIC_CEILING = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-ceiling", 1);
+    public final static FunctionIdentifier NUMERIC_FLOOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-floor", 1);
+    public final static FunctionIdentifier NUMERIC_ROUND = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "numeric-round", 1);
     public final static FunctionIdentifier NUMERIC_ROUND_HALF_TO_EVEN = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-round-half-to-even", 1);
+            FunctionConstants.ASTERIX_NS, "numeric-round-half-to-even", 1);
     public final static FunctionIdentifier NUMERIC_ROUND_HALF_TO_EVEN2 = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-round-half-to-even", 2);
+            FunctionConstants.ASTERIX_NS, "numeric-round-half-to-even", 2);
     // String funcitons
-    public final static FunctionIdentifier STRING_EQUAL = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "string-equal", 2);
-    public final static FunctionIdentifier STRING_START_WITH = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "start-with", 2);
-    public final static FunctionIdentifier STRING_END_WITH = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "end-with", 2);
-    public final static FunctionIdentifier STRING_MATCHES = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "matches", 2);
+    public final static FunctionIdentifier STRING_EQUAL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "string-equal", 2);
+    public final static FunctionIdentifier STRING_START_WITH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "start-with", 2);
+    public final static FunctionIdentifier STRING_END_WITH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "end-with", 2);
+    public final static FunctionIdentifier STRING_MATCHES = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "matches", 2);
     public final static FunctionIdentifier STRING_MATCHES_WITH_FLAG = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "matches", 3);
-    public final static FunctionIdentifier STRING_LOWERCASE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "lowercase", 1);
-    public final static FunctionIdentifier STRING_REPLACE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "replace", 3);
+            FunctionConstants.ASTERIX_NS, "matches", 3);
+    public final static FunctionIdentifier STRING_LOWERCASE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "lowercase", 1);
+    public final static FunctionIdentifier STRING_REPLACE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "replace", 3);
     public final static FunctionIdentifier STRING_REPLACE_WITH_FLAG = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "replace", 4);
-    public final static FunctionIdentifier STRING_LENGTH = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "string-length", 1);
-    public final static FunctionIdentifier SUBSTRING2 = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            FunctionConstants.ASTERIX_NS, "replace", 4);
+    public final static FunctionIdentifier STRING_LENGTH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "string-length", 1);
+    public final static FunctionIdentifier SUBSTRING2 = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "substring", 2);
-    public final static FunctionIdentifier SUBSTRING_BEFORE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "substring-before", 2);
-    public final static FunctionIdentifier SUBSTRING_AFTER = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "substring-after", 2);
-    public final static FunctionIdentifier STRING_TO_CODEPOINT = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "string-to-codepoint", 1);
-    public final static FunctionIdentifier CODEPOINT_TO_STRING = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "codepoint-to-string", 1);
-    public final static FunctionIdentifier STRING_CONCAT = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "string-concat", 1);
-    public final static FunctionIdentifier STRING_JOIN = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "string-join", 2);
+    public final static FunctionIdentifier SUBSTRING_BEFORE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "substring-before", 2);
+    public final static FunctionIdentifier SUBSTRING_AFTER = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "substring-after", 2);
+    public final static FunctionIdentifier STRING_TO_CODEPOINT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "string-to-codepoint", 1);
+    public final static FunctionIdentifier CODEPOINT_TO_STRING = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "codepoint-to-string", 1);
+    public final static FunctionIdentifier STRING_CONCAT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "string-concat", 1);
+    public final static FunctionIdentifier STRING_JOIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "string-join", 2);
 
-    public final static FunctionIdentifier DATASET = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
-            "dataset", 1);
-    public final static FunctionIdentifier FEED_INGEST = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "feed-ingest", 1);
+    public final static FunctionIdentifier DATASET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dataset", 1);
+    public final static FunctionIdentifier FEED_INGEST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "feed-ingest", 1);
 
-    public final static FunctionIdentifier INDEX_SEARCH = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "index-search", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier INDEX_SEARCH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "index-search", FunctionIdentifier.VARARGS);
 
     public final static FunctionIdentifier MAKE_FIELD_INDEX_HANDLE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "make-field-index-handle", 2);
+            FunctionConstants.ASTERIX_NS, "make-field-index-handle", 2);
     public final static FunctionIdentifier MAKE_FIELD_NAME_HANDLE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "make-field-name-handle", 1);
+            FunctionConstants.ASTERIX_NS, "make-field-name-handle", 1);
 
-    public final static FunctionIdentifier SUBSTRING = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+    public final static FunctionIdentifier SUBSTRING = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "substring", 3);
-    public final static FunctionIdentifier LIKE = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
-            "like", 2);
-    public final static FunctionIdentifier CONTAINS = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
-            "contains", 2);
-    public final static FunctionIdentifier STARTS_WITH = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "starts-with", 2);
-    public final static FunctionIdentifier ENDS_WITH = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+    public final static FunctionIdentifier LIKE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "like", 2);
+    public final static FunctionIdentifier CONTAINS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "contains",
+            2);
+    public final static FunctionIdentifier STARTS_WITH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "starts-with", 2);
+    public final static FunctionIdentifier ENDS_WITH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "ends-with", 2);
 
-    public final static FunctionIdentifier AVG = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "agg-avg", 1);
-    public final static FunctionIdentifier COUNT = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "agg-count", 1);
-    public final static FunctionIdentifier SUM = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "agg-sum", 1);
-    public final static FunctionIdentifier LOCAL_SUM = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+    public final static FunctionIdentifier AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-avg", 1);
+    public final static FunctionIdentifier COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-count", 1);
+    public final static FunctionIdentifier SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sum", 1);
+    public final static FunctionIdentifier LOCAL_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "agg-local-sum", 1);
-    public final static FunctionIdentifier MAX = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "agg-max", 1);
-    public final static FunctionIdentifier LOCAL_MAX = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+    public final static FunctionIdentifier MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-max", 1);
+    public final static FunctionIdentifier LOCAL_MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "agg-local-max", 1);
-    public final static FunctionIdentifier MIN = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "agg-min", 1);
-    public final static FunctionIdentifier LOCAL_MIN = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+    public final static FunctionIdentifier MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-min", 1);
+    public final static FunctionIdentifier LOCAL_MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "agg-local-min", 1);
-    public final static FunctionIdentifier GLOBAL_AVG = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "agg-global-avg", 1);
-    public final static FunctionIdentifier LOCAL_AVG = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+    public final static FunctionIdentifier GLOBAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "agg-global-avg", 1);
+    public final static FunctionIdentifier LOCAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "agg-local-avg", 1);
 
-    public final static FunctionIdentifier SCALAR_AVG = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
-            "avg", 1);
-    public final static FunctionIdentifier SCALAR_COUNT = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "count", 1);
-    public final static FunctionIdentifier SCALAR_SUM = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
-            "sum", 1);
-    public final static FunctionIdentifier SCALAR_MAX = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
-            "max", 1);
-    public final static FunctionIdentifier SCALAR_MIN = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
-            "min", 1);
-    public final static FunctionIdentifier SCALAR_GLOBAL_AVG = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "global-avg", 1);
-    public final static FunctionIdentifier SCALAR_LOCAL_AVG = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "local-avg", 1);
+    public final static FunctionIdentifier SCALAR_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "avg", 1);
+    public final static FunctionIdentifier SCALAR_COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "count",
+            1);
+    public final static FunctionIdentifier SCALAR_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sum", 1);
+    public final static FunctionIdentifier SCALAR_MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "max", 1);
+    public final static FunctionIdentifier SCALAR_MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "min", 1);
+    public final static FunctionIdentifier SCALAR_GLOBAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "global-avg", 1);
+    public final static FunctionIdentifier SCALAR_LOCAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "local-avg", 1);
 
     // serializable aggregate functions
-    public final static FunctionIdentifier SERIAL_AVG = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "avg-serial", 1);
-    public final static FunctionIdentifier SERIAL_COUNT = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "count-serial", 1);
-    public final static FunctionIdentifier SERIAL_SUM = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "sum-serial", 1);
-    public final static FunctionIdentifier SERIAL_LOCAL_SUM = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "local-sum-serial", 1);
-    public final static FunctionIdentifier SERIAL_GLOBAL_AVG = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "global-avg-serial", 1);
-    public final static FunctionIdentifier SERIAL_LOCAL_AVG = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "local-avg-serial", 1);
+    public final static FunctionIdentifier SERIAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "avg-serial", 1);
+    public final static FunctionIdentifier SERIAL_COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "count-serial", 1);
+    public final static FunctionIdentifier SERIAL_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "sum-serial", 1);
+    public final static FunctionIdentifier SERIAL_LOCAL_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "local-sum-serial", 1);
+    public final static FunctionIdentifier SERIAL_GLOBAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "global-avg-serial", 1);
+    public final static FunctionIdentifier SERIAL_LOCAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "local-avg-serial", 1);
 
-    public final static FunctionIdentifier SCAN_COLLECTION = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "scan-collection", 1);
-    public final static FunctionIdentifier SUBSET_COLLECTION = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "subset-collection", 3);
+    public final static FunctionIdentifier SCAN_COLLECTION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "scan-collection", 1);
+    public final static FunctionIdentifier SUBSET_COLLECTION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "subset-collection", 3);
 
-    public final static FunctionIdentifier RANGE = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
-            "range", 2);
+    public final static FunctionIdentifier RANGE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "range", 2);
 
     // fuzzy functions:
-    public final static FunctionIdentifier FUZZY_EQ = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "fuzzy-eq", 2);
+    public final static FunctionIdentifier FUZZY_EQ = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "fuzzy-eq",
+            2);
 
-    public final static FunctionIdentifier PREFIX_LEN_JACCARD = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "prefix-len-jaccard", 2);
+    public final static FunctionIdentifier PREFIX_LEN_JACCARD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "prefix-len-jaccard", 2);
 
-    public final static FunctionIdentifier SIMILARITY_JACCARD = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "similarity-jaccard", 2);
+    public final static FunctionIdentifier SIMILARITY_JACCARD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "similarity-jaccard", 2);
     public final static FunctionIdentifier SIMILARITY_JACCARD_CHECK = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "similarity-jaccard-check", 3);
+            FunctionConstants.ASTERIX_NS, "similarity-jaccard-check", 3);
     public final static FunctionIdentifier SIMILARITY_JACCARD_SORTED = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "similarity-jaccard-sorted", 2);
+            FunctionConstants.ASTERIX_NS, "similarity-jaccard-sorted", 2);
     public final static FunctionIdentifier SIMILARITY_JACCARD_SORTED_CHECK = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "similarity-jaccard-sorted-check", 3);
+            FunctionConstants.ASTERIX_NS, "similarity-jaccard-sorted-check", 3);
     public final static FunctionIdentifier SIMILARITY_JACCARD_PREFIX = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "similarity-jaccard-prefix", 6);
+            FunctionConstants.ASTERIX_NS, "similarity-jaccard-prefix", 6);
     public final static FunctionIdentifier SIMILARITY_JACCARD_PREFIX_CHECK = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "similarity-jaccard-prefix-check", 6);
+            FunctionConstants.ASTERIX_NS, "similarity-jaccard-prefix-check", 6);
 
-    public final static FunctionIdentifier EDIT_DISTANCE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "edit-distance", 2);
-    public final static FunctionIdentifier EDIT_DISTANCE_CHECK = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "edit-distance-check", 3);
+    public final static FunctionIdentifier EDIT_DISTANCE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "edit-distance", 2);
+    public final static FunctionIdentifier EDIT_DISTANCE_CHECK = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "edit-distance-check", 3);
     public final static FunctionIdentifier EDIT_DISTANCE_LIST_IS_FILTERABLE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "edit-distance-list-is-filterable", 2);
+            FunctionConstants.ASTERIX_NS, "edit-distance-list-is-filterable", 2);
     public final static FunctionIdentifier EDIT_DISTANCE_STRING_IS_FILTERABLE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "edit-distance-string-is-filterable", 4);
+            FunctionConstants.ASTERIX_NS, "edit-distance-string-is-filterable", 4);
 
     // tokenizers:
-    public final static FunctionIdentifier WORD_TOKENS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "word-tokens", 1);
-    public final static FunctionIdentifier HASHED_WORD_TOKENS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "hashed-word-tokens", 1);
+    public final static FunctionIdentifier WORD_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "word-tokens", 1);
+    public final static FunctionIdentifier HASHED_WORD_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "hashed-word-tokens", 1);
     public final static FunctionIdentifier COUNTHASHED_WORD_TOKENS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "counthashed-word-tokens", 1);
-    public final static FunctionIdentifier GRAM_TOKENS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "gram-tokens", 3);
-    public final static FunctionIdentifier HASHED_GRAM_TOKENS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "hashed-gram-tokens", 3);
+            FunctionConstants.ASTERIX_NS, "counthashed-word-tokens", 1);
+    public final static FunctionIdentifier GRAM_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "gram-tokens", 3);
+    public final static FunctionIdentifier HASHED_GRAM_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "hashed-gram-tokens", 3);
     public final static FunctionIdentifier COUNTHASHED_GRAM_TOKENS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "counthashed-gram-tokens", 3);
+            FunctionConstants.ASTERIX_NS, "counthashed-gram-tokens", 3);
 
-    public final static FunctionIdentifier TID = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "tid", 0);
+    public final static FunctionIdentifier TID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "tid", 0);
 
     // constructors:
-    public final static FunctionIdentifier BOOLEAN_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "boolean", 1);
-    public final static FunctionIdentifier NULL_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "null", 1);
-    public final static FunctionIdentifier STRING_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "string", 1);
-    public final static FunctionIdentifier INT8_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "int8", 1);
-    public final static FunctionIdentifier INT16_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "int16", 1);
-    public final static FunctionIdentifier INT32_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "int32", 1);
-    public final static FunctionIdentifier INT64_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "int64", 1);
-    public final static FunctionIdentifier FLOAT_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "float", 1);
-    public final static FunctionIdentifier DOUBLE_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "double", 1);
-    public final static FunctionIdentifier POINT_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "point", 1);
-    public final static FunctionIdentifier POINT3D_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "point3d", 1);
-    public final static FunctionIdentifier LINE_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "line", 1);
-    public final static FunctionIdentifier CIRCLE_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "circle", 1);
-    public final static FunctionIdentifier RECTANGLE_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "rectangle", 1);
-    public final static FunctionIdentifier POLYGON_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "polygon", 1);
-    public final static FunctionIdentifier TIME_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "time", 1);
-    public final static FunctionIdentifier DATE_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "date", 1);
-    public final static FunctionIdentifier DATETIME_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "datetime", 1);
-    public final static FunctionIdentifier DURATION_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "duration", 1);
+    public final static FunctionIdentifier BOOLEAN_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "boolean", 1);
+    public final static FunctionIdentifier NULL_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "null", 1);
+    public final static FunctionIdentifier STRING_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "string", 1);
+    public final static FunctionIdentifier INT8_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "int8", 1);
+    public final static FunctionIdentifier INT16_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "int16", 1);
+    public final static FunctionIdentifier INT32_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "int32", 1);
+    public final static FunctionIdentifier INT64_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "int64", 1);
+    public final static FunctionIdentifier FLOAT_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "float", 1);
+    public final static FunctionIdentifier DOUBLE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "double", 1);
+    public final static FunctionIdentifier POINT_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "point", 1);
+    public final static FunctionIdentifier POINT3D_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "point3d", 1);
+    public final static FunctionIdentifier LINE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "line", 1);
+    public final static FunctionIdentifier CIRCLE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "circle", 1);
+    public final static FunctionIdentifier RECTANGLE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "rectangle", 1);
+    public final static FunctionIdentifier POLYGON_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "polygon", 1);
+    public final static FunctionIdentifier TIME_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "time", 1);
+    public final static FunctionIdentifier DATE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "date", 1);
+    public final static FunctionIdentifier DATETIME_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "datetime", 1);
+    public final static FunctionIdentifier DURATION_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "duration", 1);
 
     public final static FunctionIdentifier YEAR_MONTH_DURATION_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "year-month-duration", 1);
+            FunctionConstants.ASTERIX_NS, "year-month-duration", 1);
     public final static FunctionIdentifier DAY_TIME_DURATION_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "day-time-duration", 1);
+            FunctionConstants.ASTERIX_NS, "day-time-duration", 1);
 
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_DATE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-from-date", 2);
+            FunctionConstants.ASTERIX_NS, "interval-from-date", 2);
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_TIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-from-time", 2);
+            FunctionConstants.ASTERIX_NS, "interval-from-time", 2);
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_DATETIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-from-datetime", 2);
+            FunctionConstants.ASTERIX_NS, "interval-from-datetime", 2);
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_START_FROM_DATE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-start-from-date", 2);
+            FunctionConstants.ASTERIX_NS, "interval-start-from-date", 2);
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_START_FROM_TIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-start-from-time", 2);
+            FunctionConstants.ASTERIX_NS, "interval-start-from-time", 2);
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_START_FROM_DATETIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-start-from-datetime", 2);
-    public final static FunctionIdentifier INTERVAL_BEFORE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-before", 2);
-    public final static FunctionIdentifier INTERVAL_AFTER = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-after", 2);
-    public final static FunctionIdentifier INTERVAL_MEETS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-meets", 2);
-    public final static FunctionIdentifier INTERVAL_MET_BY = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-met-by", 2);
-    public final static FunctionIdentifier INTERVAL_OVERLAPS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-overlaps", 2);
+            FunctionConstants.ASTERIX_NS, "interval-start-from-datetime", 2);
+    public final static FunctionIdentifier INTERVAL_BEFORE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-before", 2);
+    public final static FunctionIdentifier INTERVAL_AFTER = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-after", 2);
+    public final static FunctionIdentifier INTERVAL_MEETS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-meets", 2);
+    public final static FunctionIdentifier INTERVAL_MET_BY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-met-by", 2);
+    public final static FunctionIdentifier INTERVAL_OVERLAPS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-overlaps", 2);
     public final static FunctionIdentifier INTERVAL_OVERLAPPED_BY = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-overlapped-by", 2);
-    public final static FunctionIdentifier OVERLAP = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
-            "overlap", 2);
-    public final static FunctionIdentifier INTERVAL_STARTS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-starts", 2);
-    public final static FunctionIdentifier INTERVAL_STARTED_BY = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-started-by", 2);
-    public final static FunctionIdentifier INTERVAL_COVERS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-covers", 2);
-    public final static FunctionIdentifier INTERVAL_COVERED_BY = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-covered-by", 2);
-    public final static FunctionIdentifier INTERVAL_ENDS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-ends", 2);
-    public final static FunctionIdentifier INTERVAL_ENDED_BY = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-ended-by", 2);
-    public final static FunctionIdentifier CURRENT_TIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "current-time", 0);
-    public final static FunctionIdentifier CURRENT_DATE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "current-date", 0);
-    public final static FunctionIdentifier CURRENT_DATETIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "current-datetime", 0);
-    public final static FunctionIdentifier DURATION_EQUAL = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "duration-equal", 2);
+            FunctionConstants.ASTERIX_NS, "interval-overlapped-by", 2);
+    public final static FunctionIdentifier OVERLAP = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "overlap", 2);
+    public final static FunctionIdentifier INTERVAL_STARTS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-starts", 2);
+    public final static FunctionIdentifier INTERVAL_STARTED_BY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-started-by", 2);
+    public final static FunctionIdentifier INTERVAL_COVERS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-covers", 2);
+    public final static FunctionIdentifier INTERVAL_COVERED_BY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-covered-by", 2);
+    public final static FunctionIdentifier INTERVAL_ENDS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-ends", 2);
+    public final static FunctionIdentifier INTERVAL_ENDED_BY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "interval-ended-by", 2);
+    public final static FunctionIdentifier CURRENT_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "current-time", 0);
+    public final static FunctionIdentifier CURRENT_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "current-date", 0);
+    public final static FunctionIdentifier CURRENT_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "current-datetime", 0);
+    public final static FunctionIdentifier DURATION_EQUAL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "duration-equal", 2);
     public final static FunctionIdentifier YEAR_MONTH_DURATION_GREATER_THAN = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "year-month-duration-greater-than", 2);
+            FunctionConstants.ASTERIX_NS, "year-month-duration-greater-than", 2);
     public final static FunctionIdentifier YEAR_MONTH_DURATION_LESS_THAN = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "year-month-duration-less-than", 2);
+            FunctionConstants.ASTERIX_NS, "year-month-duration-less-than", 2);
     public final static FunctionIdentifier DAY_TIME_DURATION_GREATER_THAN = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "day-time-duration-greater-than", 2);
+            FunctionConstants.ASTERIX_NS, "day-time-duration-greater-than", 2);
     public final static FunctionIdentifier DAY_TIME_DURATION_LESS_THAN = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "day-time-duration-less-than", 2);
-    public final static FunctionIdentifier DURATION_FROM_MONTHS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "duration-from-months", 1);
+            FunctionConstants.ASTERIX_NS, "day-time-duration-less-than", 2);
+    public final static FunctionIdentifier DURATION_FROM_MONTHS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "duration-from-months", 1);
     public final static FunctionIdentifier MONTHS_FROM_YEAR_MONTH_DURATION = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "months-from-year-month-duration", 1);
+            FunctionConstants.ASTERIX_NS, "months-from-year-month-duration", 1);
     public final static FunctionIdentifier DURATION_FROM_MILLISECONDS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "duration-from-ms", 1);
+            FunctionConstants.ASTERIX_NS, "duration-from-ms", 1);
     public final static FunctionIdentifier MILLISECONDS_FROM_DAY_TIME_DURATION = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "ms-from-day-time-duration", 1);
+            FunctionConstants.ASTERIX_NS, "ms-from-day-time-duration", 1);
 
     public final static FunctionIdentifier GET_YEAR_MONTH_DURATION = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-year-month-duration", 1);
-    public final static FunctionIdentifier GET_DAY_TIME_DURATION = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-day-time-duration", 1);
+            FunctionConstants.ASTERIX_NS, "get-year-month-duration", 1);
+    public final static FunctionIdentifier GET_DAY_TIME_DURATION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "get-day-time-duration", 1);
 
     // spatial
-    public final static FunctionIdentifier CREATE_POINT = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "create-point", 2);
-    public final static FunctionIdentifier CREATE_LINE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "create-line", 2);
-    public final static FunctionIdentifier CREATE_POLYGON = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "create-polygon", FunctionIdentifier.VARARGS);
-    public final static FunctionIdentifier CREATE_CIRCLE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "create-circle", 2);
-    public final static FunctionIdentifier CREATE_RECTANGLE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "create-rectangle", 2);
-    public final static FunctionIdentifier SPATIAL_INTERSECT = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "spatial-intersect", 2);
-    public final static FunctionIdentifier SPATIAL_AREA = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "spatial-area", 1);
-    public final static FunctionIdentifier SPATIAL_DISTANCE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "spatial-distance", 2);
-    public final static FunctionIdentifier CREATE_MBR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "create-mbr", 3);
-    public final static FunctionIdentifier SPATIAL_CELL = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "spatial-cell", 4);
-    public final static FunctionIdentifier SWITCH_CASE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "switch-case", FunctionIdentifier.VARARGS);
-    public final static FunctionIdentifier REG_EXP = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "reg-exp", 2);
+    public final static FunctionIdentifier CREATE_POINT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "create-point", 2);
+    public final static FunctionIdentifier CREATE_LINE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "create-line", 2);
+    public final static FunctionIdentifier CREATE_POLYGON = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "create-polygon", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier CREATE_CIRCLE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "create-circle", 2);
+    public final static FunctionIdentifier CREATE_RECTANGLE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "create-rectangle", 2);
+    public final static FunctionIdentifier SPATIAL_INTERSECT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "spatial-intersect", 2);
+    public final static FunctionIdentifier SPATIAL_AREA = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "spatial-area", 1);
+    public final static FunctionIdentifier SPATIAL_DISTANCE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "spatial-distance", 2);
+    public final static FunctionIdentifier CREATE_MBR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "create-mbr", 3);
+    public final static FunctionIdentifier SPATIAL_CELL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "spatial-cell", 4);
+    public final static FunctionIdentifier SWITCH_CASE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "switch-case", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier REG_EXP = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reg-exp", 2);
 
-    public final static FunctionIdentifier INJECT_FAILURE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "inject-failure", 2);
-    public final static FunctionIdentifier CAST_RECORD = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PRIVATE.name(), "cast-record", 1);
-    public final static FunctionIdentifier CAST_LIST = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+    public final static FunctionIdentifier INJECT_FAILURE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "inject-failure", 2);
+    public final static FunctionIdentifier CAST_RECORD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "cast-record", 1);
+    public final static FunctionIdentifier CAST_LIST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "cast-list", 1);
 
     // Spatial and temporal type accessors
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_YEAR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "year", 1);
+            FunctionConstants.ASTERIX_NS, "year", 1);
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_MONTH = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "month", 1);
-    public static final FunctionIdentifier ACCESSOR_TEMPORAL_DAY = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "day", 1);
+            FunctionConstants.ASTERIX_NS, "month", 1);
+    public static final FunctionIdentifier ACCESSOR_TEMPORAL_DAY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "day", 1);
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_HOUR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "hour", 1);
-    public static final FunctionIdentifier ACCESSOR_TEMPORAL_MIN = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "minute", 1);
-    public static final FunctionIdentifier ACCESSOR_TEMPORAL_SEC = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "second", 1);
+            FunctionConstants.ASTERIX_NS, "hour", 1);
+    public static final FunctionIdentifier ACCESSOR_TEMPORAL_MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "minute", 1);
+    public static final FunctionIdentifier ACCESSOR_TEMPORAL_SEC = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "second", 1);
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_MILLISEC = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "millisecond", 1);
+            FunctionConstants.ASTERIX_NS, "millisecond", 1);
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_INTERVAL_START = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-interval-start", 1);
+            FunctionConstants.ASTERIX_NS, "get-interval-start", 1);
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_INTERVAL_END = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-interval-end", 1);
+            FunctionConstants.ASTERIX_NS, "get-interval-end", 1);
 
     // Temporal functions
     public static final FunctionIdentifier DATE_FROM_UNIX_TIME_IN_DAYS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "date-from-unix-time-in-days", 1);
-    public static final FunctionIdentifier DATE_FROM_DATETIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "date-from-datetime", 1);
-    public final static FunctionIdentifier ADD_DATE_DURATION = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "add-date-duration", 2);
-    public final static FunctionIdentifier SUBTRACT_DATE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "subtract-date", 2);
+            FunctionConstants.ASTERIX_NS, "date-from-unix-time-in-days", 1);
+    public static final FunctionIdentifier DATE_FROM_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "date-from-datetime", 1);
+    public final static FunctionIdentifier ADD_DATE_DURATION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "add-date-duration", 2);
+    public final static FunctionIdentifier SUBTRACT_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "subtract-date", 2);
     public final static FunctionIdentifier TIME_FROM_UNIX_TIME_IN_MS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "time-from-unix-time-in-ms", 1);
-    public final static FunctionIdentifier TIME_FROM_DATETIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "time-from-datetime", 1);
-    public final static FunctionIdentifier SUBTRACT_TIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "subtract-time", 2);
-    public final static FunctionIdentifier ADD_TIME_DURATION = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "add-time-duration", 2);
+            FunctionConstants.ASTERIX_NS, "time-from-unix-time-in-ms", 1);
+    public final static FunctionIdentifier TIME_FROM_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "time-from-datetime", 1);
+    public final static FunctionIdentifier SUBTRACT_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "subtract-time", 2);
+    public final static FunctionIdentifier ADD_TIME_DURATION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "add-time-duration", 2);
     public final static FunctionIdentifier DATETIME_FROM_UNIX_TIME_IN_MS = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "datetime-from-unix-time-in-ms", 1);
+            FunctionConstants.ASTERIX_NS, "datetime-from-unix-time-in-ms", 1);
     public final static FunctionIdentifier DATETIME_FROM_DATE_TIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "datetime-from-date-time", 2);
-    public final static FunctionIdentifier SUBTRACT_DATETIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "subtract-datetime", 2);
-    public final static FunctionIdentifier ADD_DATETIME_DURATION = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "add-datetime-duration", 2);
+            FunctionConstants.ASTERIX_NS, "datetime-from-date-time", 2);
+    public final static FunctionIdentifier SUBTRACT_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "subtract-datetime", 2);
+    public final static FunctionIdentifier ADD_DATETIME_DURATION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "add-datetime-duration", 2);
     public final static FunctionIdentifier CALENDAR_DURATION_FROM_DATETIME = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "calendar-duration-from-datetime", 2);
+            FunctionConstants.ASTERIX_NS, "calendar-duration-from-datetime", 2);
     public final static FunctionIdentifier CALENDAR_DURATION_FROM_DATE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "calendar-duration-from-date", 2);
+            FunctionConstants.ASTERIX_NS, "calendar-duration-from-date", 2);
     public final static FunctionIdentifier ADJUST_TIME_FOR_TIMEZONE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "adjust-time-for-timezone", 2);
+            FunctionConstants.ASTERIX_NS, "adjust-time-for-timezone", 2);
     public final static FunctionIdentifier ADJUST_DATETIME_FOR_TIMEZONE = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "adjust-datetime-for-timezone", 2);
+            FunctionConstants.ASTERIX_NS, "adjust-datetime-for-timezone", 2);
 
     public final static FunctionIdentifier GET_POINT_X_COORDINATE_ACCESSOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-x", 1);
+            FunctionConstants.ASTERIX_NS, "get-x", 1);
     public final static FunctionIdentifier GET_POINT_Y_COORDINATE_ACCESSOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-y", 1);
+            FunctionConstants.ASTERIX_NS, "get-y", 1);
     public final static FunctionIdentifier GET_CIRCLE_RADIUS_ACCESSOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-radius", 1);
+            FunctionConstants.ASTERIX_NS, "get-radius", 1);
     public final static FunctionIdentifier GET_CIRCLE_CENTER_ACCESSOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-center", 1);
+            FunctionConstants.ASTERIX_NS, "get-center", 1);
     public final static FunctionIdentifier GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR = new FunctionIdentifier(
-            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-points", 1);
+            FunctionConstants.ASTERIX_NS, "get-points", 1);
 
     public static final FunctionIdentifier EQ = AlgebricksBuiltinFunctions.EQ;
     public static final FunctionIdentifier LE = AlgebricksBuiltinFunctions.LE;
@@ -583,8 +561,10 @@
     public static final FunctionIdentifier NUMERIC_ADD = AlgebricksBuiltinFunctions.NUMERIC_ADD;
     public static final FunctionIdentifier IS_NULL = AlgebricksBuiltinFunctions.IS_NULL;
 
-    public static final FunctionIdentifier NOT_NULL = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
-            "not-null", 1);
+    public static final FunctionIdentifier NOT_NULL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "not-null",
+            1);
+    public static final FunctionIdentifier COLLECTION_TO_SEQUENCE = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "" + "collection-to-sequence", 1);
 
     public static IFunctionInfo getAsterixFunctionInfo(FunctionIdentifier fid) {
         IFunctionInfo finfo = registeredFunctions.get(fid);
@@ -601,72 +581,73 @@
     static {
 
         // first, take care of Algebricks builtin functions
-        add(EQ, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(LE, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(GE, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(LT, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(GT, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(IS_NULL, ABooleanTypeComputer.INSTANCE);
-        add(AND, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(NEQ, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(NOT, UnaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(OR, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(NUMERIC_ADD, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+        addFunction(IS_NULL, ABooleanTypeComputer.INSTANCE);
+        addFunction(NOT, UnaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+
+        addPrivateFunction(EQ, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addPrivateFunction(LE, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addPrivateFunction(GE, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addPrivateFunction(LT, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addPrivateFunction(GT, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addPrivateFunction(AND, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addPrivateFunction(NEQ, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addPrivateFunction(OR, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addPrivateFunction(NUMERIC_ADD, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
 
         // and then, Asterix builtin functions
-        add(NOT_NULL, NotNullTypeComputer.INSTANCE);
-        add(ANY_COLLECTION_MEMBER, NonTaggedCollectionMemberResultType.INSTANCE);
-        add(AVG, OptionalADoubleTypeComputer.INSTANCE);
-        add(BOOLEAN_CONSTRUCTOR, UnaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(CARET, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
-        add(CIRCLE_CONSTRUCTOR, OptionalACircleTypeComputer.INSTANCE);
-        add(CLOSED_RECORD_CONSTRUCTOR, ClosedRecordConstructorResultType.INSTANCE);
-        add(CONCAT_NON_NULL, ConcatNonNullTypeComputer.INSTANCE);
+        addPrivateFunction(NOT_NULL, NotNullTypeComputer.INSTANCE);
+        addPrivateFunction(ANY_COLLECTION_MEMBER, NonTaggedCollectionMemberResultType.INSTANCE);
+        addPrivateFunction(AVG, OptionalADoubleTypeComputer.INSTANCE);
+        addFunction(BOOLEAN_CONSTRUCTOR, UnaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addFunction(CARET, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+        addFunction(CIRCLE_CONSTRUCTOR, OptionalACircleTypeComputer.INSTANCE);
+        addPrivateFunction(CLOSED_RECORD_CONSTRUCTOR, ClosedRecordConstructorResultType.INSTANCE);
+        addPrivateFunction(CONCAT_NON_NULL, ConcatNonNullTypeComputer.INSTANCE);
 
-        add(CONTAINS, ABooleanTypeComputer.INSTANCE);
-        add(COUNT, AInt64TypeComputer.INSTANCE);
-        add(COUNTHASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
-        add(COUNTHASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
-        add(CREATE_CIRCLE, ACircleTypeComputer.INSTANCE);
-        add(CREATE_LINE, ALineTypeComputer.INSTANCE);
-        add(CREATE_MBR, ADoubleTypeComputer.INSTANCE);
-        add(CREATE_POINT, APointTypeComputer.INSTANCE);
-        add(CREATE_POLYGON, APolygonTypeComputer.INSTANCE);
-        add(CREATE_RECTANGLE, ARectangleTypeComputer.INSTANCE);
+        addFunction(CONTAINS, ABooleanTypeComputer.INSTANCE);
+        addPrivateFunction(COUNT, AInt64TypeComputer.INSTANCE);
+        addFunction(COUNTHASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
+        addPrivateFunction(COUNTHASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
+        addFunction(CREATE_CIRCLE, ACircleTypeComputer.INSTANCE);
+        addFunction(CREATE_LINE, ALineTypeComputer.INSTANCE);
+        addFunction(CREATE_MBR, ADoubleTypeComputer.INSTANCE);
+        addFunction(CREATE_POINT, APointTypeComputer.INSTANCE);
+        addFunction(CREATE_POLYGON, APolygonTypeComputer.INSTANCE);
+        addFunction(CREATE_RECTANGLE, ARectangleTypeComputer.INSTANCE);
 
-        add(DATE_CONSTRUCTOR, OptionalADateTypeComputer.INSTANCE);
-        add(DATETIME_CONSTRUCTOR, OptionalADateTimeTypeComputer.INSTANCE);
-        add(DOUBLE_CONSTRUCTOR, OptionalADoubleTypeComputer.INSTANCE);
-        add(DURATION_CONSTRUCTOR, OptionalADurationTypeComputer.INSTANCE);
-        add(YEAR_MONTH_DURATION_CONSTRUCTOR, OptionalAYearMonthDurationTypeComputer.INSTANCE);
-        add(DAY_TIME_DURATION_CONSTRUCTOR, OptionalADayTimeDurationTypeComputer.INSTANCE);
-        add(EDIT_DISTANCE, AInt32TypeComputer.INSTANCE);
-        add(EDIT_DISTANCE_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
-        add(EDIT_DISTANCE_STRING_IS_FILTERABLE, ABooleanTypeComputer.INSTANCE);
-        add(EDIT_DISTANCE_LIST_IS_FILTERABLE, ABooleanTypeComputer.INSTANCE);
-        add(EMBED_TYPE, new IResultTypeComputer() {
+        addFunction(DATE_CONSTRUCTOR, OptionalADateTypeComputer.INSTANCE);
+        addFunction(DATETIME_CONSTRUCTOR, OptionalADateTimeTypeComputer.INSTANCE);
+        addFunction(DOUBLE_CONSTRUCTOR, OptionalADoubleTypeComputer.INSTANCE);
+        addFunction(DURATION_CONSTRUCTOR, OptionalADurationTypeComputer.INSTANCE);
+        addFunction(YEAR_MONTH_DURATION_CONSTRUCTOR, OptionalAYearMonthDurationTypeComputer.INSTANCE);
+        addFunction(DAY_TIME_DURATION_CONSTRUCTOR, OptionalADayTimeDurationTypeComputer.INSTANCE);
+        addFunction(EDIT_DISTANCE, AInt32TypeComputer.INSTANCE);
+        addFunction(EDIT_DISTANCE_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
+        addPrivateFunction(EDIT_DISTANCE_STRING_IS_FILTERABLE, ABooleanTypeComputer.INSTANCE);
+        addPrivateFunction(EDIT_DISTANCE_LIST_IS_FILTERABLE, ABooleanTypeComputer.INSTANCE);
+        addPrivateFunction(EMBED_TYPE, new IResultTypeComputer() {
             @Override
             public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
                     IMetadataProvider<?, ?> mp) throws AlgebricksException {
                 return (IAType) BuiltinType.ANY;
             }
         });
-        add(EMPTY_STREAM, ABooleanTypeComputer.INSTANCE);
-        add(ENDS_WITH, ABooleanTypeComputer.INSTANCE);
+        addPrivateFunction(EMPTY_STREAM, ABooleanTypeComputer.INSTANCE);
+        addFunction(ENDS_WITH, ABooleanTypeComputer.INSTANCE);
         // add(FIELD_ACCESS, NonTaggedFieldAccessByNameResultType.INSTANCE);
-        add(FIELD_ACCESS_BY_INDEX, FieldAccessByIndexResultType.INSTANCE);
-        add(FIELD_ACCESS_BY_NAME, NonTaggedFieldAccessByNameResultType.INSTANCE);
-        add(FLOAT_CONSTRUCTOR, OptionalAFloatTypeComputer.INSTANCE);
-        add(FUZZY_EQ, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(GET_HANDLE, null); // TODO
-        add(GET_ITEM, NonTaggedGetItemResultType.INSTANCE);
-        add(GET_DATA, null); // TODO
-        add(GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
-        add(GRAM_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE);
-        add(GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
-        add(HASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
-        add(HASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
-        add(INDEX_SEARCH, new IResultTypeComputer() {
+        addPrivateFunction(FIELD_ACCESS_BY_INDEX, FieldAccessByIndexResultType.INSTANCE);
+        addPrivateFunction(FIELD_ACCESS_BY_NAME, NonTaggedFieldAccessByNameResultType.INSTANCE);
+        addFunction(FLOAT_CONSTRUCTOR, OptionalAFloatTypeComputer.INSTANCE);
+        addPrivateFunction(FUZZY_EQ, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addPrivateFunction(GET_HANDLE, null); // TODO
+        addPrivateFunction(GET_ITEM, NonTaggedGetItemResultType.INSTANCE);
+        addPrivateFunction(GET_DATA, null); // TODO
+        addPrivateFunction(GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
+        addPrivateFunction(GRAM_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE);
+        addFunction(GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
+        addFunction(HASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
+        addPrivateFunction(HASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
+        addPrivateFunction(INDEX_SEARCH, new IResultTypeComputer() {
 
             @Override
             public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
@@ -674,95 +655,96 @@
                 return BuiltinType.ANY; // TODO
             }
         });
-        add(INT8_CONSTRUCTOR, OptionalAInt8TypeComputer.INSTANCE);
-        add(INT16_CONSTRUCTOR, OptionalAInt16TypeComputer.INSTANCE);
-        add(INT32_CONSTRUCTOR, OptionalAInt32TypeComputer.INSTANCE);
-        add(INT64_CONSTRUCTOR, OptionalAInt64TypeComputer.INSTANCE);
-        add(LEN, OptionalAInt32TypeComputer.INSTANCE);
-        add(LIKE, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
-        add(LINE_CONSTRUCTOR, OptionalALineTypeComputer.INSTANCE);
-        add(LISTIFY, OrderedListConstructorResultType.INSTANCE);
-        add(LOCAL_AVG, NonTaggedLocalAvgTypeComputer.INSTANCE);
-        add(MAKE_FIELD_INDEX_HANDLE, null); // TODO
-        add(MAKE_FIELD_NAME_HANDLE, null); // TODO
-        add(MAX, NonTaggedSumTypeComputer.INSTANCE);
-        add(LOCAL_MAX, NonTaggedSumTypeComputer.INSTANCE);
-        add(MIN, NonTaggedSumTypeComputer.INSTANCE);
-        add(LOCAL_MIN, NonTaggedSumTypeComputer.INSTANCE);
-        add(NON_EMPTY_STREAM, ABooleanTypeComputer.INSTANCE);
-        add(NULL_CONSTRUCTOR, ANullTypeComputer.INSTANCE);
-        add(NUMERIC_UNARY_MINUS, NonTaggedUnaryMinusTypeComputer.INSTANCE);
-        add(NUMERIC_SUBTRACT, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
-        add(NUMERIC_MULTIPLY, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
-        add(NUMERIC_DIVIDE, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
-        add(NUMERIC_MOD, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
-        add(NUMERIC_IDIV, AInt32TypeComputer.INSTANCE);
+        addFunction(INT8_CONSTRUCTOR, OptionalAInt8TypeComputer.INSTANCE);
+        addFunction(INT16_CONSTRUCTOR, OptionalAInt16TypeComputer.INSTANCE);
+        addFunction(INT32_CONSTRUCTOR, OptionalAInt32TypeComputer.INSTANCE);
+        addFunction(INT64_CONSTRUCTOR, OptionalAInt64TypeComputer.INSTANCE);
+        addFunction(LEN, OptionalAInt32TypeComputer.INSTANCE);
+        addFunction(LIKE, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
+        addFunction(LINE_CONSTRUCTOR, OptionalALineTypeComputer.INSTANCE);
+        addPrivateFunction(LISTIFY, OrderedListConstructorResultType.INSTANCE);
+        addPrivateFunction(LOCAL_AVG, NonTaggedLocalAvgTypeComputer.INSTANCE);
+        addPrivateFunction(MAKE_FIELD_INDEX_HANDLE, null); // TODO
+        addPrivateFunction(MAKE_FIELD_NAME_HANDLE, null); // TODO
+        addPrivateFunction(MAX, NonTaggedSumTypeComputer.INSTANCE);
+        addPrivateFunction(LOCAL_MAX, NonTaggedSumTypeComputer.INSTANCE);
+        addPrivateFunction(MIN, NonTaggedSumTypeComputer.INSTANCE);
+        addPrivateFunction(LOCAL_MIN, NonTaggedSumTypeComputer.INSTANCE);
+        addPrivateFunction(NON_EMPTY_STREAM, ABooleanTypeComputer.INSTANCE);
+        addFunction(NULL_CONSTRUCTOR, ANullTypeComputer.INSTANCE);
+        addFunction(NUMERIC_UNARY_MINUS, NonTaggedUnaryMinusTypeComputer.INSTANCE);
+        addFunction(NUMERIC_SUBTRACT, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+        addFunction(NUMERIC_MULTIPLY, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+        addFunction(NUMERIC_DIVIDE, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+        addFunction(NUMERIC_MOD, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+        addFunction(NUMERIC_IDIV, AInt32TypeComputer.INSTANCE);
 
-        add(NUMERIC_ABS, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
-        add(NUMERIC_CEILING, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
-        add(NUMERIC_FLOOR, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
-        add(NUMERIC_ROUND, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
-        add(NUMERIC_ROUND_HALF_TO_EVEN, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
-        add(NUMERIC_ROUND_HALF_TO_EVEN2, NonTaggedNumericRoundHalfToEven2TypeComputer.INSTANCE);
+        addFunction(NUMERIC_ABS, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
+        addFunction(NUMERIC_CEILING, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
+        addFunction(NUMERIC_FLOOR, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
+        addFunction(NUMERIC_ROUND, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
+        addFunction(NUMERIC_ROUND_HALF_TO_EVEN, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
+        addFunction(NUMERIC_ROUND_HALF_TO_EVEN2, NonTaggedNumericRoundHalfToEven2TypeComputer.INSTANCE);
 
-        add(STRING_TO_CODEPOINT, OrderedListOfAInt32TypeComputer.INSTANCE);
-        add(CODEPOINT_TO_STRING, AStringTypeComputer.INSTANCE);
-        add(STRING_CONCAT, OptionalAStringTypeComputer.INSTANCE);
-        add(SUBSTRING2, Substring2TypeComputer.INSTANCE);
-        add(STRING_LENGTH, UnaryStringInt32OrNullTypeComputer.INSTANCE);
-        add(STRING_LOWERCASE, UnaryStringOrNullTypeComputer.INSTANCE);
-        add(STRING_START_WITH, BinaryStringBoolOrNullTypeComputer.INSTANCE);
-        add(STRING_END_WITH, BinaryStringBoolOrNullTypeComputer.INSTANCE);
-        add(STRING_MATCHES, BinaryStringBoolOrNullTypeComputer.INSTANCE);
-        add(STRING_MATCHES_WITH_FLAG, TripleStringBoolOrNullTypeComputer.INSTANCE);
-        add(STRING_REPLACE, TripleStringStringOrNullTypeComputer.INSTANCE);
-        add(STRING_REPLACE_WITH_FLAG, QuadStringStringOrNullTypeComputer.INSTANCE);
-        add(SUBSTRING_BEFORE, BinaryStringStringOrNullTypeComputer.INSTANCE);
-        add(SUBSTRING_AFTER, BinaryStringStringOrNullTypeComputer.INSTANCE);
-        add(STRING_EQUAL, BinaryStringBoolOrNullTypeComputer.INSTANCE);
-        add(STRING_JOIN, AStringTypeComputer.INSTANCE);
+        addFunction(STRING_TO_CODEPOINT, OrderedListOfAInt32TypeComputer.INSTANCE);
+        addFunction(CODEPOINT_TO_STRING, AStringTypeComputer.INSTANCE);
+        addFunction(STRING_CONCAT, OptionalAStringTypeComputer.INSTANCE);
+        addFunction(SUBSTRING2, Substring2TypeComputer.INSTANCE);
+        addFunction(STRING_LENGTH, UnaryStringInt32OrNullTypeComputer.INSTANCE);
+        addFunction(STRING_LOWERCASE, UnaryStringOrNullTypeComputer.INSTANCE);
+        addFunction(STRING_START_WITH, BinaryStringBoolOrNullTypeComputer.INSTANCE);
+        addFunction(STRING_END_WITH, BinaryStringBoolOrNullTypeComputer.INSTANCE);
+        addFunction(STRING_MATCHES, BinaryStringBoolOrNullTypeComputer.INSTANCE);
+        addFunction(STRING_MATCHES_WITH_FLAG, TripleStringBoolOrNullTypeComputer.INSTANCE);
+        addFunction(STRING_REPLACE, TripleStringStringOrNullTypeComputer.INSTANCE);
+        addFunction(STRING_REPLACE_WITH_FLAG, QuadStringStringOrNullTypeComputer.INSTANCE);
+        addFunction(SUBSTRING_BEFORE, BinaryStringStringOrNullTypeComputer.INSTANCE);
+        addFunction(SUBSTRING_AFTER, BinaryStringStringOrNullTypeComputer.INSTANCE);
+        addPrivateFunction(STRING_EQUAL, BinaryStringBoolOrNullTypeComputer.INSTANCE);
+        addFunction(STRING_JOIN, AStringTypeComputer.INSTANCE);
 
-        add(OPEN_RECORD_CONSTRUCTOR, OpenRecordConstructorResultType.INSTANCE);
-        add(ORDERED_LIST_CONSTRUCTOR, OrderedListConstructorResultType.INSTANCE);
-        add(POINT_CONSTRUCTOR, OptionalAPointTypeComputer.INSTANCE);
-        add(POINT3D_CONSTRUCTOR, OptionalAPoint3DTypeComputer.INSTANCE);
-        add(POLYGON_CONSTRUCTOR, OptionalAPolygonTypeComputer.INSTANCE);
-        add(PREFIX_LEN_JACCARD, AInt32TypeComputer.INSTANCE);
-        add(RANGE, AInt32TypeComputer.INSTANCE);
-        add(RECTANGLE_CONSTRUCTOR, OptionalARectangleTypeComputer.INSTANCE);
+        addPrivateFunction(OPEN_RECORD_CONSTRUCTOR, OpenRecordConstructorResultType.INSTANCE);
+        addPrivateFunction(ORDERED_LIST_CONSTRUCTOR, OrderedListConstructorResultType.INSTANCE);
+        addFunction(POINT_CONSTRUCTOR, OptionalAPointTypeComputer.INSTANCE);
+        addFunction(POINT3D_CONSTRUCTOR, OptionalAPoint3DTypeComputer.INSTANCE);
+        addFunction(POLYGON_CONSTRUCTOR, OptionalAPolygonTypeComputer.INSTANCE);
+        addPrivateFunction(PREFIX_LEN_JACCARD, AInt32TypeComputer.INSTANCE);
+        addFunction(RANGE, AInt32TypeComputer.INSTANCE);
+        addFunction(RECTANGLE_CONSTRUCTOR, OptionalARectangleTypeComputer.INSTANCE);
         // add(RECORD_TYPE_CONSTRUCTOR, null);
-        add(SCALAR_AVG, ScalarVersionOfAggregateResultType.INSTANCE);
-        add(SCALAR_COUNT, AInt64TypeComputer.INSTANCE);
-        add(SCALAR_GLOBAL_AVG, ScalarVersionOfAggregateResultType.INSTANCE);
-        add(SCALAR_LOCAL_AVG, ScalarVersionOfAggregateResultType.INSTANCE);
-        add(SCALAR_MAX, ScalarVersionOfAggregateResultType.INSTANCE);
-        add(SCALAR_MIN, ScalarVersionOfAggregateResultType.INSTANCE);
-        add(SCALAR_SUM, ScalarVersionOfAggregateResultType.INSTANCE);
-        add(SCAN_COLLECTION, NonTaggedCollectionMemberResultType.INSTANCE);
-        add(SERIAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
-        add(SERIAL_COUNT, AInt64TypeComputer.INSTANCE);
-        add(SERIAL_GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
-        add(SERIAL_LOCAL_AVG, NonTaggedLocalAvgTypeComputer.INSTANCE);
-        add(SERIAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
-        add(SERIAL_LOCAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
-        add(SIMILARITY_JACCARD, AFloatTypeComputer.INSTANCE);
-        add(SIMILARITY_JACCARD_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
-        add(SIMILARITY_JACCARD_SORTED, AFloatTypeComputer.INSTANCE);
-        add(SIMILARITY_JACCARD_SORTED_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
-        add(SIMILARITY_JACCARD_PREFIX, AFloatTypeComputer.INSTANCE);
-        add(SIMILARITY_JACCARD_PREFIX_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
-        add(SPATIAL_AREA, ADoubleTypeComputer.INSTANCE);
-        add(SPATIAL_CELL, ARectangleTypeComputer.INSTANCE);
-        add(SPATIAL_DISTANCE, ADoubleTypeComputer.INSTANCE);
-        add(SPATIAL_INTERSECT, ABooleanTypeComputer.INSTANCE);
-        add(GET_POINT_X_COORDINATE_ACCESSOR, ADoubleTypeComputer.INSTANCE);
-        add(GET_POINT_Y_COORDINATE_ACCESSOR, ADoubleTypeComputer.INSTANCE);
-        add(GET_CIRCLE_RADIUS_ACCESSOR, ADoubleTypeComputer.INSTANCE);
-        add(GET_CIRCLE_CENTER_ACCESSOR, APointTypeComputer.INSTANCE);
-        add(GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR, OrderedListOfAPointTypeComputer.INSTANCE);
-        add(STARTS_WITH, ABooleanTypeComputer.INSTANCE);
-        add(STRING_CONSTRUCTOR, OptionalAStringTypeComputer.INSTANCE);
-        add(SUBSET_COLLECTION, new IResultTypeComputer() {
+
+        addFunction(SCALAR_AVG, ScalarVersionOfAggregateResultType.INSTANCE);
+        addFunction(SCALAR_COUNT, AInt64TypeComputer.INSTANCE);
+        addPrivateFunction(SCALAR_GLOBAL_AVG, ScalarVersionOfAggregateResultType.INSTANCE);
+        addPrivateFunction(SCALAR_LOCAL_AVG, ScalarVersionOfAggregateResultType.INSTANCE);
+        addFunction(SCALAR_MAX, ScalarVersionOfAggregateResultType.INSTANCE);
+        addFunction(SCALAR_MIN, ScalarVersionOfAggregateResultType.INSTANCE);
+        addFunction(SCALAR_SUM, ScalarVersionOfAggregateResultType.INSTANCE);
+        addPrivateFunction(SCAN_COLLECTION, NonTaggedCollectionMemberResultType.INSTANCE);
+        addPrivateFunction(SERIAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
+        addPrivateFunction(SERIAL_COUNT, AInt64TypeComputer.INSTANCE);
+        addPrivateFunction(SERIAL_GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
+        addPrivateFunction(SERIAL_LOCAL_AVG, NonTaggedLocalAvgTypeComputer.INSTANCE);
+        addPrivateFunction(SERIAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
+        addPrivateFunction(SERIAL_LOCAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
+        addFunction(SIMILARITY_JACCARD, AFloatTypeComputer.INSTANCE);
+        addFunction(SIMILARITY_JACCARD_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
+        addPrivateFunction(SIMILARITY_JACCARD_SORTED, AFloatTypeComputer.INSTANCE);
+        addPrivateFunction(SIMILARITY_JACCARD_SORTED_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
+        addPrivateFunction(SIMILARITY_JACCARD_PREFIX, AFloatTypeComputer.INSTANCE);
+        addPrivateFunction(SIMILARITY_JACCARD_PREFIX_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
+        addFunction(SPATIAL_AREA, ADoubleTypeComputer.INSTANCE);
+        addFunction(SPATIAL_CELL, ARectangleTypeComputer.INSTANCE);
+        addFunction(SPATIAL_DISTANCE, ADoubleTypeComputer.INSTANCE);
+        addFunction(SPATIAL_INTERSECT, ABooleanTypeComputer.INSTANCE);
+        addFunction(GET_POINT_X_COORDINATE_ACCESSOR, ADoubleTypeComputer.INSTANCE);
+        addFunction(GET_POINT_Y_COORDINATE_ACCESSOR, ADoubleTypeComputer.INSTANCE);
+        addFunction(GET_CIRCLE_RADIUS_ACCESSOR, ADoubleTypeComputer.INSTANCE);
+        addFunction(GET_CIRCLE_CENTER_ACCESSOR, APointTypeComputer.INSTANCE);
+        addFunction(GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR, OrderedListOfAPointTypeComputer.INSTANCE);
+        addFunction(STARTS_WITH, ABooleanTypeComputer.INSTANCE);
+        addFunction(STRING_CONSTRUCTOR, OptionalAStringTypeComputer.INSTANCE);
+        addPrivateFunction(SUBSET_COLLECTION, new IResultTypeComputer() {
 
             @Override
             public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
@@ -799,20 +781,20 @@
                 }
             }
         });
-        add(SUBSTRING, SubstringTypeComputer.INSTANCE);
-        add(SUM, NonTaggedSumTypeComputer.INSTANCE);
-        add(LOCAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
-        add(SWITCH_CASE, NonTaggedSwitchCaseComputer.INSTANCE);
-        add(REG_EXP, ABooleanTypeComputer.INSTANCE);
-        add(INJECT_FAILURE, InjectFailureTypeComputer.INSTANCE);
-        add(CAST_RECORD, CastRecordResultTypeComputer.INSTANCE);
-        add(CAST_LIST, CastListResultTypeComputer.INSTANCE);
+        addFunction(SUBSTRING, SubstringTypeComputer.INSTANCE);
+        addPrivateFunction(SUM, NonTaggedSumTypeComputer.INSTANCE);
+        addPrivateFunction(LOCAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
+        addFunction(SWITCH_CASE, NonTaggedSwitchCaseComputer.INSTANCE);
+        addPrivateFunction(REG_EXP, ABooleanTypeComputer.INSTANCE);
+        addFunction(INJECT_FAILURE, InjectFailureTypeComputer.INSTANCE);
+        addPrivateFunction(CAST_RECORD, CastRecordResultTypeComputer.INSTANCE);
+        addFunction(CAST_LIST, CastListResultTypeComputer.INSTANCE);
 
-        add(TID, AInt32TypeComputer.INSTANCE);
-        add(TIME_CONSTRUCTOR, OptionalATimeTypeComputer.INSTANCE);
-        add(TYPE_OF, null); // TODO
-        add(UNORDERED_LIST_CONSTRUCTOR, UnorderedListConstructorResultType.INSTANCE);
-        add(WORD_TOKENS, new IResultTypeComputer() {
+        addFunction(TID, AInt32TypeComputer.INSTANCE);
+        addFunction(TIME_CONSTRUCTOR, OptionalATimeTypeComputer.INSTANCE);
+        addFunction(TYPE_OF, null); // TODO
+        addPrivateFunction(UNORDERED_LIST_CONSTRUCTOR, UnorderedListConstructorResultType.INSTANCE);
+        addFunction(WORD_TOKENS, new IResultTypeComputer() {
 
             @Override
             public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
@@ -822,68 +804,70 @@
         });
 
         // temporal type accessors
-        add(ACCESSOR_TEMPORAL_YEAR, OptionalAInt32TypeComputer.INSTANCE);
-        add(ACCESSOR_TEMPORAL_MONTH, OptionalAInt32TypeComputer.INSTANCE);
-        add(ACCESSOR_TEMPORAL_DAY, OptionalAInt32TypeComputer.INSTANCE);
-        add(ACCESSOR_TEMPORAL_HOUR, OptionalAInt32TypeComputer.INSTANCE);
-        add(ACCESSOR_TEMPORAL_MIN, OptionalAInt32TypeComputer.INSTANCE);
-        add(ACCESSOR_TEMPORAL_SEC, OptionalAInt32TypeComputer.INSTANCE);
-        add(ACCESSOR_TEMPORAL_MILLISEC, OptionalAInt32TypeComputer.INSTANCE);
-        add(ACCESSOR_TEMPORAL_INTERVAL_START, OptionalATemporalInstanceTypeComputer.INSTANCE);
-        add(ACCESSOR_TEMPORAL_INTERVAL_END, OptionalATemporalInstanceTypeComputer.INSTANCE);
+        addFunction(ACCESSOR_TEMPORAL_YEAR, OptionalAInt32TypeComputer.INSTANCE);
+        addFunction(ACCESSOR_TEMPORAL_MONTH, OptionalAInt32TypeComputer.INSTANCE);
+        addFunction(ACCESSOR_TEMPORAL_DAY, OptionalAInt32TypeComputer.INSTANCE);
+        addFunction(ACCESSOR_TEMPORAL_HOUR, OptionalAInt32TypeComputer.INSTANCE);
+        addFunction(ACCESSOR_TEMPORAL_MIN, OptionalAInt32TypeComputer.INSTANCE);
+        addFunction(ACCESSOR_TEMPORAL_SEC, OptionalAInt32TypeComputer.INSTANCE);
+        addFunction(ACCESSOR_TEMPORAL_MILLISEC, OptionalAInt32TypeComputer.INSTANCE);
+        addFunction(ACCESSOR_TEMPORAL_INTERVAL_START, OptionalATemporalInstanceTypeComputer.INSTANCE);
+        addFunction(ACCESSOR_TEMPORAL_INTERVAL_END, OptionalATemporalInstanceTypeComputer.INSTANCE);
 
         // temporal functions
-        add(DATE_FROM_UNIX_TIME_IN_DAYS, OptionalADateTypeComputer.INSTANCE);
-        add(DATE_FROM_DATETIME, OptionalADateTypeComputer.INSTANCE);
-        add(ADD_DATE_DURATION, OptionalADateTypeComputer.INSTANCE);
-        add(SUBTRACT_DATE, OptionalADurationTypeComputer.INSTANCE);
-        add(TIME_FROM_UNIX_TIME_IN_MS, OptionalATimeTypeComputer.INSTANCE);
-        add(TIME_FROM_DATETIME, OptionalATimeTypeComputer.INSTANCE);
-        add(SUBTRACT_TIME, OptionalADurationTypeComputer.INSTANCE);
-        add(ADD_TIME_DURATION, OptionalATimeTypeComputer.INSTANCE);
-        add(DATETIME_FROM_DATE_TIME, OptionalADateTimeTypeComputer.INSTANCE);
-        add(DATETIME_FROM_UNIX_TIME_IN_MS, OptionalADateTimeTypeComputer.INSTANCE);
-        add(SUBTRACT_DATETIME, OptionalADurationTypeComputer.INSTANCE);
-        add(ADD_DATETIME_DURATION, OptionalADateTimeTypeComputer.INSTANCE);
-        add(CALENDAR_DURATION_FROM_DATETIME, OptionalADurationTypeComputer.INSTANCE);
-        add(CALENDAR_DURATION_FROM_DATE, OptionalADurationTypeComputer.INSTANCE);
-        add(ADJUST_DATETIME_FOR_TIMEZONE, OptionalAStringTypeComputer.INSTANCE);
-        add(ADJUST_TIME_FOR_TIMEZONE, OptionalAStringTypeComputer.INSTANCE);
-        add(INTERVAL_BEFORE, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_AFTER, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_MEETS, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_MET_BY, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_OVERLAPS, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_OVERLAPPED_BY, OptionalABooleanTypeComputer.INSTANCE);
-        add(OVERLAP, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_STARTS, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_STARTED_BY, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_COVERS, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_COVERED_BY, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_ENDS, OptionalABooleanTypeComputer.INSTANCE);
-        add(INTERVAL_ENDED_BY, OptionalABooleanTypeComputer.INSTANCE);
-        add(CURRENT_DATE, ADateTypeComputer.INSTANCE);
-        add(CURRENT_TIME, ATimeTypeComputer.INSTANCE);
-        add(CURRENT_DATETIME, ADateTimeTypeComputer.INSTANCE);
-        add(DAY_TIME_DURATION_GREATER_THAN, OptionalABooleanTypeComputer.INSTANCE);
-        add(DAY_TIME_DURATION_LESS_THAN, OptionalABooleanTypeComputer.INSTANCE);
-        add(YEAR_MONTH_DURATION_GREATER_THAN, OptionalABooleanTypeComputer.INSTANCE);
-        add(YEAR_MONTH_DURATION_LESS_THAN, OptionalABooleanTypeComputer.INSTANCE);
-        add(DURATION_EQUAL, OptionalABooleanTypeComputer.INSTANCE);
-        add(DURATION_FROM_MONTHS, OptionalADurationTypeComputer.INSTANCE);
-        add(DURATION_FROM_MILLISECONDS, OptionalADurationTypeComputer.INSTANCE);
-        add(MONTHS_FROM_YEAR_MONTH_DURATION, OptionalAInt32TypeComputer.INSTANCE);
-        add(MILLISECONDS_FROM_DAY_TIME_DURATION, OptionalAInt64TypeComputer.INSTANCE);
-        add(GET_DAY_TIME_DURATION, OptionalADayTimeDurationTypeComputer.INSTANCE);
-        add(GET_YEAR_MONTH_DURATION, OptionalAYearMonthDurationTypeComputer.INSTANCE);
+        addFunction(DATE_FROM_UNIX_TIME_IN_DAYS, OptionalADateTypeComputer.INSTANCE);
+        addFunction(DATE_FROM_DATETIME, OptionalADateTypeComputer.INSTANCE);
+        addFunction(ADD_DATE_DURATION, OptionalADateTypeComputer.INSTANCE);
+        addFunction(SUBTRACT_DATE, OptionalADurationTypeComputer.INSTANCE);
+        addFunction(TIME_FROM_UNIX_TIME_IN_MS, OptionalATimeTypeComputer.INSTANCE);
+        addFunction(TIME_FROM_DATETIME, OptionalATimeTypeComputer.INSTANCE);
+        addFunction(SUBTRACT_TIME, OptionalADurationTypeComputer.INSTANCE);
+        addFunction(ADD_TIME_DURATION, OptionalATimeTypeComputer.INSTANCE);
+        addFunction(DATETIME_FROM_DATE_TIME, OptionalADateTimeTypeComputer.INSTANCE);
+        addFunction(DATETIME_FROM_UNIX_TIME_IN_MS, OptionalADateTimeTypeComputer.INSTANCE);
+        addFunction(SUBTRACT_DATETIME, OptionalADurationTypeComputer.INSTANCE);
+        addFunction(ADD_DATETIME_DURATION, OptionalADateTimeTypeComputer.INSTANCE);
+        addFunction(CALENDAR_DURATION_FROM_DATETIME, OptionalADurationTypeComputer.INSTANCE);
+        addFunction(CALENDAR_DURATION_FROM_DATE, OptionalADurationTypeComputer.INSTANCE);
+        addFunction(ADJUST_DATETIME_FOR_TIMEZONE, OptionalAStringTypeComputer.INSTANCE);
+        addFunction(ADJUST_TIME_FOR_TIMEZONE, OptionalAStringTypeComputer.INSTANCE);
+        addFunction(INTERVAL_BEFORE, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_AFTER, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_MEETS, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_MET_BY, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_OVERLAPS, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_OVERLAPPED_BY, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(OVERLAP, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_STARTS, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_STARTED_BY, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_COVERS, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_COVERED_BY, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_ENDS, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(INTERVAL_ENDED_BY, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(CURRENT_DATE, ADateTypeComputer.INSTANCE);
+        addFunction(CURRENT_TIME, ATimeTypeComputer.INSTANCE);
+        addFunction(CURRENT_DATETIME, ADateTimeTypeComputer.INSTANCE);
+        addFunction(DAY_TIME_DURATION_GREATER_THAN, OptionalABooleanTypeComputer.INSTANCE);
+        addPrivateFunction(DAY_TIME_DURATION_LESS_THAN, OptionalABooleanTypeComputer.INSTANCE);
+        addPrivateFunction(YEAR_MONTH_DURATION_GREATER_THAN, OptionalABooleanTypeComputer.INSTANCE);
+        addPrivateFunction(YEAR_MONTH_DURATION_LESS_THAN, OptionalABooleanTypeComputer.INSTANCE);
+        addPrivateFunction(DURATION_EQUAL, OptionalABooleanTypeComputer.INSTANCE);
+        addFunction(DURATION_FROM_MONTHS, OptionalADurationTypeComputer.INSTANCE);
+        addFunction(DURATION_FROM_MILLISECONDS, OptionalADurationTypeComputer.INSTANCE);
+        addFunction(MONTHS_FROM_YEAR_MONTH_DURATION, OptionalAInt32TypeComputer.INSTANCE);
+        addFunction(MILLISECONDS_FROM_DAY_TIME_DURATION, OptionalAInt64TypeComputer.INSTANCE);
+        addFunction(GET_DAY_TIME_DURATION, OptionalADayTimeDurationTypeComputer.INSTANCE);
+        addFunction(GET_YEAR_MONTH_DURATION, OptionalAYearMonthDurationTypeComputer.INSTANCE);
 
         // interval constructors
-        add(INTERVAL_CONSTRUCTOR_DATE, OptionalAIntervalTypeComputer.INSTANCE);
-        add(INTERVAL_CONSTRUCTOR_TIME, OptionalAIntervalTypeComputer.INSTANCE);
-        add(INTERVAL_CONSTRUCTOR_DATETIME, OptionalAIntervalTypeComputer.INSTANCE);
-        add(INTERVAL_CONSTRUCTOR_START_FROM_DATE, OptionalAIntervalTypeComputer.INSTANCE);
-        add(INTERVAL_CONSTRUCTOR_START_FROM_DATETIME, OptionalAIntervalTypeComputer.INSTANCE);
-        add(INTERVAL_CONSTRUCTOR_START_FROM_TIME, OptionalAIntervalTypeComputer.INSTANCE);
+        addFunction(INTERVAL_CONSTRUCTOR_DATE, OptionalAIntervalTypeComputer.INSTANCE);
+        addFunction(INTERVAL_CONSTRUCTOR_TIME, OptionalAIntervalTypeComputer.INSTANCE);
+        addFunction(INTERVAL_CONSTRUCTOR_DATETIME, OptionalAIntervalTypeComputer.INSTANCE);
+        addFunction(INTERVAL_CONSTRUCTOR_START_FROM_DATE, OptionalAIntervalTypeComputer.INSTANCE);
+        addFunction(INTERVAL_CONSTRUCTOR_START_FROM_DATETIME, OptionalAIntervalTypeComputer.INSTANCE);
+        addFunction(INTERVAL_CONSTRUCTOR_START_FROM_TIME, OptionalAIntervalTypeComputer.INSTANCE);
+
+        addPrivateFunction(COLLECTION_TO_SEQUENCE, CollectionToSequenceTypeComputer.INSTANCE);
 
         String metadataFunctionLoaderClassName = "edu.uci.ics.asterix.metadata.functions.MetadataBuiltinFunctions";
         try {
@@ -977,8 +961,27 @@
         return datasetFunctions.contains(getAsterixFunctionInfo(fi));
     }
 
-    public static boolean isBuiltinCompilerFunction(FunctionIdentifier fi) {
-        return builtinFunctionsSet.keySet().contains(getAsterixFunctionInfo(fi));
+    public static boolean isBuiltinCompilerFunction(FunctionIdentifier fi, boolean includePrivateFunctions) {
+        return builtinPublicFunctionsSet.keySet().contains(getAsterixFunctionInfo(fi));
+    }
+
+    public static boolean isBuiltinCompilerFunction(FunctionSignature signature, boolean includePrivateFunctions) {
+
+        FunctionIdentifier fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, signature.getName(),
+                signature.getArity());
+        IFunctionInfo finfo = getAsterixFunctionInfo(fi);
+        if (builtinPublicFunctionsSet.keySet().contains(finfo)
+                || (includePrivateFunctions && builtinPrivateFunctionsSet.keySet().contains(finfo))) {
+            return true;
+        }
+        fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, signature.getName(), signature.getArity());
+        finfo = getAsterixFunctionInfo(fi);
+        if (builtinPublicFunctionsSet.keySet().contains(finfo)
+                || (includePrivateFunctions && builtinPrivateFunctionsSet.keySet().contains(finfo))) {
+            return true;
+        }
+
+        return false;
     }
 
     public static boolean isBuiltinAggregateFunction(FunctionIdentifier fi) {
@@ -1063,9 +1066,16 @@
         return finfo == null ? null : finfo.getFunctionIdentifier();
     }
 
-    public static void add(FunctionIdentifier fi, IResultTypeComputer typeComputer) {
+    public static void addFunction(FunctionIdentifier fi, IResultTypeComputer typeComputer) {
         IFunctionInfo functionInfo = getAsterixFunctionInfo(fi);
-        builtinFunctionsSet.put(functionInfo, functionInfo);
+        builtinPublicFunctionsSet.put(functionInfo, functionInfo);
+        funTypeComputer.put(functionInfo, typeComputer);
+        registeredFunctions.put(fi);
+    }
+
+    public static void addPrivateFunction(FunctionIdentifier fi, IResultTypeComputer typeComputer) {
+        IFunctionInfo functionInfo = getAsterixFunctionInfo(fi);
+        builtinPrivateFunctionsSet.put(functionInfo, functionInfo);
         funTypeComputer.put(functionInfo, typeComputer);
         registeredFunctions.put(fi);
     }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/TypeComputerUtilities.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/TypeComputerUtilities.java
index 2c7ff10..84d177f 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/TypeComputerUtilities.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/base/TypeComputerUtilities.java
@@ -23,9 +23,9 @@
     public static boolean setRequiredAndInputTypes(AbstractFunctionCallExpression expr, IAType requiredRecordType,
             IAType inputRecordType) {
         boolean changed = false;
-        Object opaqueParameter = expr.getOpaqueParameters();
-        if (opaqueParameter == null) {
-            Object[] opaqueParameters = new Object[2];
+        Object[] opaqueParameters = expr.getOpaqueParameters();
+        if (opaqueParameters == null) {
+            opaqueParameters = new Object[2];
             opaqueParameters[0] = requiredRecordType;
             opaqueParameters[1] = inputRecordType;
             expr.setOpaqueParameters(opaqueParameters);
@@ -34,6 +34,10 @@
         return changed;
     }
 
+    public static void resetRequiredAndInputTypes(AbstractFunctionCallExpression expr) {
+        expr.setOpaqueParameters(null);
+    }
+
     public static IAType getRequiredType(AbstractFunctionCallExpression expr) {
         Object[] type = expr.getOpaqueParameters();
         if (type != null) {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CollectionToSequenceTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CollectionToSequenceTypeComputer.java
new file mode 100644
index 0000000..5066d4b
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CollectionToSequenceTypeComputer.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeHelper;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+/**
+ * This function is to make a sequence of records and a singleton collection of records
+ * present in a uniformed manner.
+ * 
+ * @author yingyib
+ */
+public class CollectionToSequenceTypeComputer implements IResultTypeComputer {
+
+    public static final CollectionToSequenceTypeComputer INSTANCE = new CollectionToSequenceTypeComputer();
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression func = (AbstractFunctionCallExpression) expression;
+        ILogicalExpression arg = func.getArguments().get(0).getValue();
+
+        IAType argType = (IAType) env.getType(arg);
+        if (TypeHelper.canBeNull(argType)) {
+            IAType nonOptionalType = TypeHelper.getNonOptionalType(argType);
+            if (nonOptionalType.getTypeTag() == ATypeTag.ORDEREDLIST
+                    || nonOptionalType.getTypeTag() == ATypeTag.UNORDEREDLIST) {
+                /** if the collection is null, that corresponds to an empty sequence */
+                argType = nonOptionalType;
+            }
+        }
+
+        ATypeTag argTypeTag = argType.getTypeTag();
+        if (argTypeTag == ATypeTag.ORDEREDLIST || argTypeTag == ATypeTag.UNORDEREDLIST) {
+            /** if the input is a singleton list, return it's item type if any */
+            AbstractCollectionType collectionType = (AbstractCollectionType) argType;
+            return collectionType.getItemType();
+        } else {
+            /** if the input is not a singleton list, return the original input type */
+            return argType;
+        }
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/UnaryStringInt32OrNullTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/UnaryStringInt32OrNullTypeComputer.java
index 2d3018d..7b82c41 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/UnaryStringInt32OrNullTypeComputer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/UnaryStringInt32OrNullTypeComputer.java
@@ -1,6 +1,5 @@
 package edu.uci.ics.asterix.om.typecomputer.impl;
 
-
 import java.util.ArrayList;
 import java.util.List;
 
@@ -17,20 +16,21 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
 /**
- *
  * @author Xiaoyu Ma
  */
-public class UnaryStringInt32OrNullTypeComputer implements IResultTypeComputer  {   
-    
+public class UnaryStringInt32OrNullTypeComputer implements IResultTypeComputer {
+
     public static final UnaryStringInt32OrNullTypeComputer INSTANCE = new UnaryStringInt32OrNullTypeComputer();
-    private UnaryStringInt32OrNullTypeComputer() {}
-    
+
+    private UnaryStringInt32OrNullTypeComputer() {
+    }
+
     @Override
     public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
             IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
         AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
-        if(fce.getArguments().isEmpty())
-            throw new AlgebricksException("Wrong Argument Number.");        
+        if (fce.getArguments().isEmpty())
+            throw new AlgebricksException("Wrong Argument Number.");
         ILogicalExpression arg0 = fce.getArguments().get(0).getValue();
         IAType t0;
         try {
@@ -38,21 +38,23 @@
         } catch (AlgebricksException e) {
             throw new AlgebricksException(e);
         }
-        if (t0.getTypeTag() != ATypeTag.NULL &&
-            t0.getTypeTag() != ATypeTag.STRING ) {
+        if (t0.getTypeTag() != ATypeTag.NULL
+                && t0.getTypeTag() != ATypeTag.STRING
+                && (t0.getTypeTag() == ATypeTag.UNION && !((AUnionType) t0).getUnionList()
+                        .contains(BuiltinType.ASTRING))) {
             throw new NotImplementedException("Expects String Type.");
-        }     
-        
+        }
+
         List<IAType> unionList = new ArrayList<IAType>();
         unionList.add(BuiltinType.ANULL);
-        if(t0.getTypeTag() == ATypeTag.NULL) {
+        if (t0.getTypeTag() == ATypeTag.NULL) { 
             return BuiltinType.ANULL;
         }
-        
-        if(t0.getTypeTag() == ATypeTag.STRING) {
+
+        if (t0.getTypeTag() == ATypeTag.STRING || t0.getTypeTag().equals(ATypeTag.UNION)) {
             unionList.add(BuiltinType.AINT32);
-        }        
-        
+        }
+
         return new AUnionType(unionList, "String-length-Result");
-    }      
+    }
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
index 037cd29..7d20886 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
@@ -254,6 +254,8 @@
                 throw new AlgebricksException("A field with this name  \"" + fieldName + "\" could not be found.");
             }
             switch (fieldType.getTypeTag()) {
+                case INT8:
+                case INT16:
                 case INT32:
                 case INT64:
                 case FLOAT:
@@ -305,6 +307,8 @@
                         case TIME:
                         case DATETIME:
                         case UNION:
+                        case YEARMONTHDURATION:
+                        case DAYTIMEDURATION:
                             break;
                         default:
                             throw new AlgebricksException("The field \"" + fieldName + "\" which is of type "
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
similarity index 96%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfo.java
rename to asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
index 06a1c1f..81100bf 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfo.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.common.api;
+package edu.uci.ics.asterix.om.util;
 
 import java.util.logging.Logger;
 
@@ -23,9 +23,9 @@
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
-import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -57,7 +57,6 @@
         INSTANCE.metadataProperties = new AsterixMetadataProperties(propertiesAccessor);
         INSTANCE.storageProperties = new AsterixStorageProperties(propertiesAccessor);
         INSTANCE.txnProperties = new AsterixTransactionProperties(propertiesAccessor);
-
         Logger.getLogger("edu.uci.ics").setLevel(INSTANCE.externalProperties.getLogLevel());
     }
 
@@ -70,21 +69,11 @@
     }
 
     @Override
-    public IStorageManagerInterface getStorageManagerInterface() {
-        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
-    }
-
-    @Override
     public ICCApplicationContext getCCApplicationContext() {
         return appCtx;
     }
 
     @Override
-    public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
-        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
-    }
-
-    @Override
     public AsterixStorageProperties getStorageProperties() {
         return storageProperties;
     }
@@ -108,4 +97,14 @@
     public AsterixExternalProperties getExternalProperties() {
         return externalProperties;
     }
+
+    @Override
+    public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
+        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+    }
+
+    @Override
+    public IStorageManagerInterface getStorageManagerInterface() {
+        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+    }
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
new file mode 100644
index 0000000..bc2a6af
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixClusterProperties.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.util;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * A holder class for properties related to the Asterix cluster.
+ */
+
+public class AsterixClusterProperties {
+
+    private static final Logger LOGGER = Logger.getLogger(AsterixClusterProperties.class.getName());
+
+    private static final String IO_DEVICES = "iodevices";
+
+    public static final AsterixClusterProperties INSTANCE = new AsterixClusterProperties();
+
+    private Map<String, Map<String, String>> ncConfiguration = new HashMap<String, Map<String, String>>();
+
+    private AsterixClusterProperties() {
+    }
+
+    public enum State {
+        ACTIVE,
+        UNUSABLE
+    }
+
+    private State state = State.UNUSABLE;
+
+    public void removeNCConfiguration(String nodeId) {
+        state = State.UNUSABLE;
+        ncConfiguration.remove(nodeId);
+    }
+
+    public void addNCConfiguration(String nodeId, Map<String, String> configuration) {
+        ncConfiguration.put(nodeId, configuration);
+        if (ncConfiguration.keySet().size() == AsterixAppContextInfo.getInstance().getMetadataProperties()
+                .getNodeNames().size()) {
+            state = State.ACTIVE;
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(" Registering configuration parameters for node id" + nodeId);
+        }
+    }
+
+    /**
+     * Returns the number of IO devices configured for a Node Controller
+     * 
+     * @param nodeId
+     *            unique identifier of the Node Controller
+     * @return number of IO devices. -1 if the node id is not valid. A node id is not valid
+     *         if it does not correspond to the set of registered Node Controllers.
+     */
+    public int getNumberOfIODevices(String nodeId) {
+        Map<String, String> ncConfig = ncConfiguration.get(nodeId);
+        if (ncConfig == null) {
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                LOGGER.warning("Configuration parameters for nodeId" + nodeId
+                        + " not found. The node has not joined yet or has left.");
+            }
+            return -1;
+        }
+        return ncConfig.get(IO_DEVICES).split(",").length;
+    }
+
+    public State getState() {
+        return state;
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixRuntimeUtil.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixRuntimeUtil.java
index e280b2e..75fc5a7 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixRuntimeUtil.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixRuntimeUtil.java
@@ -23,46 +23,38 @@
 import java.util.Map;
 import java.util.Set;
 
-import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
-
 /**
  * Utility class for obtaining information on the set of Hyracks NodeController
  * processes that are running on a given host.
  */
 public class AsterixRuntimeUtil {
 
-	public static Set<String> getNodeControllersOnIP(String ipAddress)
-			throws Exception {
-		Map<String, Set<String>> nodeControllerInfo = getNodeControllerMap();
-		Set<String> nodeControllersAtLocation = nodeControllerInfo
-				.get(ipAddress);
-		return nodeControllersAtLocation;
-	}
+    public static Set<String> getNodeControllersOnIP(String ipAddress) throws Exception {
+        Map<String, Set<String>> nodeControllerInfo = getNodeControllerMap();
+        Set<String> nodeControllersAtLocation = nodeControllerInfo.get(ipAddress);
+        return nodeControllersAtLocation;
+    }
 
-	public static List<String> getAllNodeControllers() throws Exception {
-		Collection<Set<String>> nodeControllersCollection = getNodeControllerMap()
-				.values();
-		List<String> nodeControllers = new ArrayList<String>();
-		for (Set<String> ncCollection : nodeControllersCollection) {
-			nodeControllers.addAll(ncCollection);
-		}
-		return nodeControllers;
-	}
+    public static List<String> getAllNodeControllers() throws Exception {
+        Collection<Set<String>> nodeControllersCollection = getNodeControllerMap().values();
+        List<String> nodeControllers = new ArrayList<String>();
+        for (Set<String> ncCollection : nodeControllersCollection) {
+            nodeControllers.addAll(ncCollection);
+        }
+        return nodeControllers;
+    }
 
-	public static Map<String, Set<String>> getNodeControllerMap()
-			throws Exception {
-		Map<String, Set<String>> map = new HashMap<String, Set<String>>();
-		AsterixAppContextInfo.getInstance().getCCApplicationContext()
-				.getCCContext().getIPAddressNodeMap(map);
-		return map;
-	}
+    public static Map<String, Set<String>> getNodeControllerMap() throws Exception {
+        Map<String, Set<String>> map = new HashMap<String, Set<String>>();
+        AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext().getIPAddressNodeMap(map);
+        return map;
+    }
 
-	public static String getIPAddress(String hostname)
-			throws UnknownHostException {
-		String address = InetAddress.getByName(hostname).getHostAddress();
-		if (address.equals("127.0.1.1")) {
-			address = "127.0.0.1";
-		}
-		return address;
-	}
+    public static String getIPAddress(String hostname) throws UnknownHostException {
+        String address = InetAddress.getByName(hostname).getHostAddress();
+        if (address.equals("127.0.1.1")) {
+            address = "127.0.0.1";
+        }
+        return address;
+    }
 }
diff --git a/asterix-runtime/.gitignore b/asterix-runtime/.gitignore
new file mode 100644
index 0000000..ea8c4bf
--- /dev/null
+++ b/asterix-runtime/.gitignore
@@ -0,0 +1 @@
+/target
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AbstractAsterixListIterator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AbstractAsterixListIterator.java
index efae5f9..269d363 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AbstractAsterixListIterator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AbstractAsterixListIterator.java
@@ -13,7 +13,10 @@
     protected byte[] data;
     protected int count = 0;
     protected int pos = -1;
-    protected int size = -1;
+    protected int nextPos = -1;
+    protected int itemLen = -1;
+    protected int numberOfItems = -1;
+    protected int listLength = -1;
     protected int startOff = -1;
     protected IBinaryComparator cmp;
 
@@ -27,12 +30,12 @@
 
     @Override
     public boolean hasNext() {
-        return count < size;
+        return count < numberOfItems;
     }
 
     @Override
     public int size() {
-        return size;
+        return numberOfItems;
     }
 
     @Override
@@ -45,10 +48,20 @@
         return pos;
     }
 
+    public int getItemLen() {
+        return itemLen;
+    }
+
     @Override
     public void next() {
         try {
-            pos = getItemOffset(data, startOff, ++count);
+            pos = nextPos;
+            ++count;
+            nextPos = startOff + listLength;
+            if (count + 1 < numberOfItems) {
+                nextPos = getItemOffset(data, startOff, count + 1);
+            }
+            itemLen = nextPos - pos;
         } catch (AsterixException e) {
             throw new AsterixRuntimeException(e);
         }
@@ -59,6 +72,11 @@
         count = 0;
         try {
             pos = getItemOffset(data, startOff, count);
+            nextPos = startOff + listLength;
+            if (count + 1 < numberOfItems) {
+                nextPos = getItemOffset(data, startOff, count + 1);
+            }
+            itemLen = nextPos - pos;
         } catch (AsterixException e) {
             throw new AsterixRuntimeException(e);
         }
@@ -67,7 +85,8 @@
     public void reset(byte[] data, int startOff) {
         this.data = data;
         this.startOff = startOff;
-        size = getNumberOfItems(data, startOff);
+        this.numberOfItems = getNumberOfItems(data, startOff);
+        this.listLength = getListLength(data, startOff);
         ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(data[startOff + 1]);
         switch (tag) {
             case INT32: {
@@ -102,4 +121,6 @@
     protected abstract int getItemOffset(byte[] serOrderedList, int offset, int itemIndex) throws AsterixException;
 
     protected abstract int getNumberOfItems(byte[] serOrderedList, int offset);
+
+    protected abstract int getListLength(byte[] serOrderedList, int offset);
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AsterixOrderedListIterator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AsterixOrderedListIterator.java
index d3714c1..fc92875 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AsterixOrderedListIterator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AsterixOrderedListIterator.java
@@ -14,4 +14,9 @@
     protected int getNumberOfItems(byte[] serOrderedList, int offset) {
         return AOrderedListSerializerDeserializer.getNumberOfItems(serOrderedList, offset);
     }
+
+    @Override
+    protected int getListLength(byte[] serOrderedList, int offset) {
+        return AOrderedListSerializerDeserializer.getOrderedListLength(serOrderedList, offset + 1);
+    }
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AsterixUnorderedListIterator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AsterixUnorderedListIterator.java
index de7742b..5f01581 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AsterixUnorderedListIterator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/AsterixUnorderedListIterator.java
@@ -14,4 +14,9 @@
     protected int getNumberOfItems(byte[] serOrderedList, int offset) {
         return AUnorderedListSerializerDeserializer.getNumberOfItems(serOrderedList, offset);
     }
+
+    @Override
+    protected int getListLength(byte[] serOrderedList, int offset) {
+        return AUnorderedListSerializerDeserializer.getUnorderedListLength(serOrderedList, offset + 1);
+    }
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
index ab73df2..b732f40 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
@@ -62,8 +62,8 @@
             probeListCount++;
             byte[] buf = probeIter.getData();
             int off = probeIter.getPos();
-            int len = getItemLen(buf, off);
-            keyEntry.set(buf, off, len);            
+            int len = probeIter.getItemLen();
+            keyEntry.set(buf, off, len);
             BinaryEntry entry = hashMap.get(keyEntry);
             if (entry != null) {
                 // Increment second value.
@@ -94,7 +94,7 @@
         }
         return intersectionSize;
     }
-    
+
     @Override
     protected void writeResult(float jacc) throws IOException {
         listBuilder.reset(listType);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java
index 9f5c9c8..391fb30 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardEvaluator.java
@@ -4,8 +4,6 @@
 import java.io.IOException;
 import java.util.Arrays;
 
-import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
-import edu.uci.ics.asterix.formats.nontagged.AqlBinaryHashFunctionFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.om.base.AFloat;
 import edu.uci.ics.asterix.om.base.AMutableFloat;
@@ -22,9 +20,10 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ListItemBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.hash.ListItemBinaryHashFunctionFactory;
 
 public class SimilarityJaccardEvaluator implements ICopyEvaluator {
 
@@ -58,7 +57,8 @@
     protected int firstStart = -1;
     protected int secondStart = -1;
     protected float jaccSim = 0.0f;
-    protected ATypeTag itemTypeTag;
+    protected ATypeTag firstItemTypeTag;
+    protected ATypeTag secondItemTypeTag;
 
     protected BinaryHashMap hashMap;
     protected BinaryEntry keyEntry = new BinaryEntry();
@@ -105,6 +105,9 @@
 
         firstTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[firstStart]);
         secondTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[secondStart]);
+
+        firstItemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[firstStart + 1]);
+        secondItemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[secondStart + 1]);
     }
 
     protected boolean prepareLists(byte[] bytes, int firstStart, int secondStart, ATypeTag argType)
@@ -116,17 +119,12 @@
         if (firstListIter.size() == 0 || secondListIter.size() == 0) {
             return false;
         }
-        if (firstTypeTag == ATypeTag.ANY || secondTypeTag == ATypeTag.ANY) {
-            throw new AlgebricksException("\n Jaccard can only be called on homogenous lists");
-        }
         // TODO: Check item types are compatible.
-        itemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[firstStart + 1]);
         return true;
     }
 
     protected float computeResult(byte[] bytes, int firstStart, int secondStart, ATypeTag argType)
             throws AlgebricksException {
-        setHashMap(bytes, firstStart, secondStart);
         // We will subtract the intersection size later to get the real union size.
         int firstListSize = firstListIter.size();
         int secondListSize = secondListIter.size();
@@ -136,7 +134,10 @@
         AbstractAsterixListIterator probeList = (buildList == firstListIter) ? secondListIter : firstListIter;
         int buildListSize = (buildList == firstListIter) ? firstListSize : secondListSize;
         int probeListSize = (probeList == firstListIter) ? firstListSize : secondListSize;
+        ATypeTag buildItemTypeTag = (buildList == firstListIter) ? firstItemTypeTag : secondItemTypeTag;
+        ATypeTag probeItemTypeTag = (probeList == firstListIter) ? firstItemTypeTag : secondItemTypeTag;
 
+        setHashMap(bytes, buildItemTypeTag, probeItemTypeTag);
         buildHashMap(buildList);
         int intersectionSize = probeHashMap(probeList, buildListSize, probeListSize);
         // Special indicator for the "check" version of jaccard.
@@ -154,7 +155,7 @@
         while (buildIter.hasNext()) {
             byte[] buf = buildIter.getData();
             int off = buildIter.getPos();
-            int len = getItemLen(buf, off);
+            int len = buildIter.getItemLen();
             keyEntry.set(buf, off, len);
             BinaryEntry entry = hashMap.put(keyEntry, valEntry);
             if (entry != null) {
@@ -172,7 +173,7 @@
         while (probeIter.hasNext()) {
             byte[] buf = probeIter.getData();
             int off = probeIter.getPos();
-            int len = getItemLen(buf, off);
+            int len = probeIter.getItemLen();
             keyEntry.set(buf, off, len);
             BinaryEntry entry = hashMap.get(keyEntry);
             if (entry != null) {
@@ -195,69 +196,19 @@
         return intersectionSize;
     }
 
-    protected void setHashMap(byte[] bytes, int firstStart, int secondStart) {
+    protected void setHashMap(byte[] bytes, ATypeTag buildItemTypeTag, ATypeTag probeItemTypeTag) {
         if (hashMap != null) {
             hashMap.clear();
             return;
         }
-        IBinaryHashFunction hashFunc = null;
-        IBinaryComparator cmp = null;
-        switch (itemTypeTag) {
-            case INT32: {
-                hashFunc = AqlBinaryHashFunctionFactoryProvider.INTEGER_POINTABLE_INSTANCE.createBinaryHashFunction();
-                cmp = AqlBinaryComparatorFactoryProvider.INTEGER_POINTABLE_INSTANCE.createBinaryComparator();
-                break;
-            }
-            case FLOAT: {
-                hashFunc = AqlBinaryHashFunctionFactoryProvider.FLOAT_POINTABLE_INSTANCE.createBinaryHashFunction();
-                cmp = AqlBinaryComparatorFactoryProvider.FLOAT_POINTABLE_INSTANCE.createBinaryComparator();
-                break;
-            }
-            case DOUBLE: {
-                hashFunc = AqlBinaryHashFunctionFactoryProvider.DOUBLE_POINTABLE_INSTANCE.createBinaryHashFunction();
-                cmp = AqlBinaryComparatorFactoryProvider.DOUBLE_POINTABLE_INSTANCE.createBinaryComparator();
-                break;
-            }
-            case STRING: {
-                if (ignoreCase) {
-                    // Ignore case in comparisons and hashing.
-                    hashFunc = AqlBinaryHashFunctionFactoryProvider.UTF8STRING_LOWERCASE_POINTABLE_INSTANCE
-                            .createBinaryHashFunction();
-                    cmp = AqlBinaryComparatorFactoryProvider.UTF8STRING_LOWERCASE_POINTABLE_INSTANCE
-                            .createBinaryComparator();
-                } else {
-                    hashFunc = AqlBinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE
-                            .createBinaryHashFunction();
-                    cmp = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
-                }
-                break;
-            }
-            default: {
-                break;
-            }
-        }
-        hashMap = new BinaryHashMap(TABLE_SIZE, TABLE_FRAME_SIZE, hashFunc, cmp);
-    }
 
-    protected int getItemLen(byte[] bytes, int itemOff) {
-        switch (itemTypeTag) {
-            case INT32: {
-                return 4;
-            }
-            case FLOAT: {
-                return 4;
-            }
-            case DOUBLE: {
-                return 8;
-            }
-            case STRING: {
-                // 2 bytes for the UTF8 len, plus the string data.
-                return 2 + UTF8StringPointable.getUTFLength(bytes, itemOff);
-            }
-            default: {
-                return -1;
-            }
-        }
+        IBinaryHashFunction putHashFunc = ListItemBinaryHashFunctionFactory.INSTANCE.createBinaryHashFunction(
+                buildItemTypeTag, ignoreCase);
+        IBinaryHashFunction getHashFunc = ListItemBinaryHashFunctionFactory.INSTANCE.createBinaryHashFunction(
+                probeItemTypeTag, ignoreCase);
+        IBinaryComparator cmp = ListItemBinaryComparatorFactory.INSTANCE.createBinaryComparator(buildItemTypeTag,
+                probeItemTypeTag, ignoreCase);
+        hashMap = new BinaryHashMap(TABLE_SIZE, TABLE_FRAME_SIZE, putHashFunc, getHashFunc, cmp);
     }
 
     protected boolean checkArgTypes(ATypeTag typeTag1, ATypeTag typeTag2) throws AlgebricksException {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
index ce9ecd2..2bd874f 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
@@ -6,7 +6,6 @@
 
 import java.io.DataOutput;
 import java.util.Arrays;
-import java.util.regex.Pattern;
 
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.om.base.AMutableString;
@@ -24,9 +23,6 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-/**
- * @author ilovesoup
- */
 public abstract class AbstractQuadStringStringEval implements ICopyEvaluator {
 
     private DataOutput dout;
@@ -138,7 +134,7 @@
             } else if (c == '_') {
                 sb.append(".");
             } else {
-                if (Arrays.binarySearch(reservedRegexChars, c) >= 0) {
+                if (Arrays.binarySearch(StringEvaluatorUtils.reservedRegexChars, c) >= 0) {
                     sb.append('\\');
                 }
                 sb.append(c);
@@ -147,33 +143,4 @@
         return sb.toString();
     }
 
-    protected int toFlag(AString pattern) {
-        String str = pattern.getStringValue();
-        int flag = 0;
-        for (int i = 0; i < str.length(); i++) {
-            char c = str.charAt(i);
-            switch (c) {
-                case 's':
-                    flag |= Pattern.DOTALL;
-                    break;
-                case 'm':
-                    flag |= Pattern.MULTILINE;
-                    break;
-                case 'i':
-                    flag |= Pattern.CASE_INSENSITIVE;
-                    break;
-                case 'x':
-                    flag |= Pattern.COMMENTS;
-                    break;
-            }
-        }
-        return flag;
-    }
-
-    private final static char[] reservedRegexChars = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$',
-            '*', '|' };
-
-    static {
-        Arrays.sort(reservedRegexChars);
-    }
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
index a3be797..7d0a01f 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
@@ -2,7 +2,6 @@
 
 import java.io.DataOutput;
 import java.util.Arrays;
-import java.util.regex.Pattern;
 
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.om.base.ABoolean;
@@ -19,9 +18,6 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-/**
- * @author Xiaoyu Ma
- */
 public abstract class AbstractTripleStringBoolEval implements ICopyEvaluator {
 
     private DataOutput dout;
@@ -118,7 +114,7 @@
             } else if (c == '_') {
                 sb.append(".");
             } else {
-                if (Arrays.binarySearch(reservedRegexChars, c) >= 0) {
+                if (Arrays.binarySearch(StringEvaluatorUtils.reservedRegexChars, c) >= 0) {
                     sb.append('\\');
                 }
                 sb.append(c);
@@ -126,34 +122,4 @@
         }
         return sb.toString();
     }
-
-    protected int toFlag(AString pattern) {
-        String str = pattern.getStringValue();
-        int flag = 0;
-        for (int i = 0; i < str.length(); i++) {
-            char c = str.charAt(i);
-            switch (c) {
-                case 's':
-                    flag |= Pattern.DOTALL;
-                    break;
-                case 'm':
-                    flag |= Pattern.MULTILINE;
-                    break;
-                case 'i':
-                    flag |= Pattern.CASE_INSENSITIVE;
-                    break;
-                case 'x':
-                    flag |= Pattern.COMMENTS;
-                    break;
-            }
-        }
-        return flag;
-    }
-
-    private final static char[] reservedRegexChars = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$',
-            '*', '|' };
-
-    static {
-        Arrays.sort(reservedRegexChars);
-    }
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
index 41d0b3d..4833f16 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
@@ -20,9 +20,6 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-/**
- * @author Xiaoyu Ma
- */
 public abstract class AbstractTripleStringStringEval implements ICopyEvaluator {
 
     private DataOutput dout;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/BinaryHashMap.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/BinaryHashMap.java
index 240f8c7..6367996 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/BinaryHashMap.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/BinaryHashMap.java
@@ -20,150 +20,154 @@
  * Intended to work with binary data and be able to map arbitrary key types to
  * arbitrary value types, given that they have implementations of
  * IBinaryHashFunction and IBinaryComparator.
- * 
  * Uses 2 bytes each to indicate the length of the key and the value.
  * Uses 8 byte pointers for the linked list (4 bytes frame index, 4 bytes frame offset).
- * 
  * This class is NOT thread safe.
- * 
  */
 public class BinaryHashMap {
-	// Special value to indicate an empty "bucket" in the header array.
-	private static final long NULL_PTR = -1;
-	private static final int PTR_SIZE = 8;
-	private static final int SLOT_SIZE = 2;
-	private static final int ENTRY_HEADER_SIZE = PTR_SIZE + 2 * SLOT_SIZE;
-	private final IBinaryHashFunction hashFunc;
-	private final IBinaryComparator cmp;
-	private final BinaryEntry returnValue = new BinaryEntry();
-	
-	private final long[] listHeads;
-	private final int frameSize;
-	private final List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
-	private int currFrameIndex;
-	private int nextOff;
-	private int size;
-	
-	// Can be used for key or value.
-	public static class BinaryEntry {
-		public byte[] buf;
-		public int off;
-		public int len;
-		
-		public void set(byte[] buf, int off, int len) {
-			this.buf = buf;
-			this.off = off;
-			this.len = len;
-		}
-		
-		// Inefficient. Just for debugging.
-		@SuppressWarnings("rawtypes")
-		public String print(ISerializerDeserializer serde) throws HyracksDataException {
-			ByteArrayInputStream inStream = new ByteArrayInputStream(buf, off, len);
+    // Special value to indicate an empty "bucket" in the header array.
+    private static final long NULL_PTR = -1;
+    private static final int PTR_SIZE = 8;
+    private static final int SLOT_SIZE = 2;
+    private static final int ENTRY_HEADER_SIZE = PTR_SIZE + 2 * SLOT_SIZE;
+    private final IBinaryHashFunction putHashFunc;
+    private final IBinaryHashFunction getHashFunc;
+    private final IBinaryComparator cmp;
+    private final BinaryEntry returnValue = new BinaryEntry();
+
+    private final long[] listHeads;
+    private final int frameSize;
+    private final List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
+    private int currFrameIndex;
+    private int nextOff;
+    private int size;
+
+    // Can be used for key or value.
+    public static class BinaryEntry {
+        public byte[] buf;
+        public int off;
+        public int len;
+
+        public void set(byte[] buf, int off, int len) {
+            this.buf = buf;
+            this.off = off;
+            this.len = len;
+        }
+
+        // Inefficient. Just for debugging.
+        @SuppressWarnings("rawtypes")
+        public String print(ISerializerDeserializer serde) throws HyracksDataException {
+            ByteArrayInputStream inStream = new ByteArrayInputStream(buf, off, len);
             DataInput dataIn = new DataInputStream(inStream);
             return serde.deserialize(dataIn).toString();
-		}
-	}
-	
-	public BinaryHashMap(int tableSize, int frameSize, IBinaryHashFunction hashFunc, IBinaryComparator cmp) {
-		listHeads = new long[tableSize];
-		this.frameSize = frameSize;		
-		this.hashFunc = hashFunc;
-		this.cmp = cmp;
-		frames.add(ByteBuffer.allocate(frameSize));
-		clear();
-	}
-	
-	/**
-	 * Inserts key, value into the hash map. If key already exists, returns
-	 * existing entry. Otherwise, returns null.
-	 * 
-	 * @param key
-	 * @param value
-	 * @return
-	 */
-	public BinaryEntry put(BinaryEntry key, BinaryEntry value) {
-		return getPutInternal(key, value, true);
-	}
-	
-	/**
-	 * Retrieves value for given key. Returns null if key doesn't exist.
-	 * 
-	 * @param key
-	 * @param value
-	 * @return
-	 */
-	public BinaryEntry get(BinaryEntry key) {
-		return getPutInternal(key, null, false);
-	}
-	
-	private BinaryEntry getPutInternal(BinaryEntry key, BinaryEntry value, boolean put) {
-		int bucket = Math.abs(hashFunc.hash(key.buf, key.off, key.len) % listHeads.length);
-		long headPtr = listHeads[bucket];
-		if (headPtr == NULL_PTR) {
-			// Key definitely doesn't exist yet.
-			if (put) {
-				listHeads[bucket] = appendEntry(key, value);
-			}
-			return null;
-		}
-		// Follow the chain until we found an entry matching the given key.
-		int frameOff;
-		ByteBuffer frame;
-		do {				
-			int frameIndex = getFrameIndex(headPtr);
-			frameOff = getFrameOffset(headPtr);
-			frame = frames.get(frameIndex);
-			int entryKeyOff = frameOff + ENTRY_HEADER_SIZE;
-			int entryKeyLen = frame.getShort(frameOff);
-			if (cmp.compare(frame.array(), entryKeyOff, entryKeyLen, key.buf,
-					key.off, key.len) == 0) {
-				// Key found, set values and return.
-				int entryValOff = frameOff + ENTRY_HEADER_SIZE + entryKeyLen;
-				int entryValLen = frame.getShort(frameOff + SLOT_SIZE);
-				returnValue.set(frame.array(), entryValOff, entryValLen);
-				return returnValue;
-			}
-			headPtr = frame.getLong(frameOff + 2 * SLOT_SIZE);
-		} while (headPtr != NULL_PTR);
-		// We've followed the chain to its end, and didn't find the key.
-		if (put) {
-			// Append the new entry, and set a pointer to it in the last entry we've checked.
-			long newPtr = appendEntry(key, value);
-			frame.putLong(frameOff + 2 * SLOT_SIZE, newPtr);
-		}
-		return null;
-	}
-	
-	public long appendEntry(BinaryEntry key, BinaryEntry value) {
-		ByteBuffer frame = frames.get(currFrameIndex);
-		int requiredSpace = key.len + value.len + ENTRY_HEADER_SIZE;
-		if (nextOff + requiredSpace >= frameSize) {
-			// Entry doesn't fit on frame, allocate a new one.
-			if (requiredSpace > frameSize) {
-				throw new IllegalStateException("Key and value greater than framesize.");
-			}
-			frames.add(ByteBuffer.allocate(frameSize));
-			currFrameIndex++;
-			nextOff = 0;
-			frame = frames.get(currFrameIndex);
-		}
-		writeEntryHeader(frame, nextOff, key.len, value.len, NULL_PTR);
-		System.arraycopy(key.buf, key.off, frame.array(), nextOff + ENTRY_HEADER_SIZE, key.len);
-		System.arraycopy(value.buf, value.off, frame.array(), nextOff + ENTRY_HEADER_SIZE + key.len, value.len);
-		long entryPtr = getEntryPtr(currFrameIndex, nextOff);
-		nextOff += requiredSpace;
-		size++;
-		return entryPtr;
-	}
-	
-	private void writeEntryHeader(ByteBuffer frame, int targetOff, int keyLen, int valLen, long ptr) {
-		frame.putShort(targetOff, (short) keyLen);
-		frame.putShort(targetOff + SLOT_SIZE, (short) valLen);
-		frame.putLong(targetOff + 2 * SLOT_SIZE, ptr);
-	}
+        }
+    }
 
-	private long getEntryPtr(int frameIndex, int frameOff) {
+    public BinaryHashMap(int tableSize, int frameSize, IBinaryHashFunction putHashFunc,
+            IBinaryHashFunction getHashFunc, IBinaryComparator cmp) {
+        listHeads = new long[tableSize];
+        this.frameSize = frameSize;
+        this.putHashFunc = putHashFunc;
+        this.getHashFunc = getHashFunc;
+        this.cmp = cmp;
+        frames.add(ByteBuffer.allocate(frameSize));
+        clear();
+    }
+
+    /**
+     * Inserts key, value into the hash map. If key already exists, returns
+     * existing entry. Otherwise, returns null.
+     * 
+     * @param key
+     * @param value
+     * @return
+     */
+    public BinaryEntry put(BinaryEntry key, BinaryEntry value) {
+        return getPutInternal(key, value, true);
+    }
+
+    /**
+     * Retrieves value for given key. Returns null if key doesn't exist.
+     * 
+     * @param key
+     * @param value
+     * @return
+     */
+    public BinaryEntry get(BinaryEntry key) {
+        return getPutInternal(key, null, false);
+    }
+
+    private BinaryEntry getPutInternal(BinaryEntry key, BinaryEntry value, boolean put) {
+        int bucket;
+        if (put) {
+            bucket = Math.abs(putHashFunc.hash(key.buf, key.off, key.len) % listHeads.length);
+        } else {
+            bucket = Math.abs(getHashFunc.hash(key.buf, key.off, key.len) % listHeads.length);
+        }
+        long headPtr = listHeads[bucket];
+        if (headPtr == NULL_PTR) {
+            // Key definitely doesn't exist yet.
+            if (put) {
+                listHeads[bucket] = appendEntry(key, value);
+            }
+            return null;
+        }
+        // Follow the chain until we found an entry matching the given key.
+        int frameOff;
+        ByteBuffer frame;
+        do {
+            int frameIndex = getFrameIndex(headPtr);
+            frameOff = getFrameOffset(headPtr);
+            frame = frames.get(frameIndex);
+            int entryKeyOff = frameOff + ENTRY_HEADER_SIZE;
+            int entryKeyLen = frame.getShort(frameOff);
+            if (cmp.compare(frame.array(), entryKeyOff, entryKeyLen, key.buf, key.off, key.len) == 0) {
+                // Key found, set values and return.
+                int entryValOff = frameOff + ENTRY_HEADER_SIZE + entryKeyLen;
+                int entryValLen = frame.getShort(frameOff + SLOT_SIZE);
+                returnValue.set(frame.array(), entryValOff, entryValLen);
+                return returnValue;
+            }
+            headPtr = frame.getLong(frameOff + 2 * SLOT_SIZE);
+        } while (headPtr != NULL_PTR);
+        // We've followed the chain to its end, and didn't find the key.
+        if (put) {
+            // Append the new entry, and set a pointer to it in the last entry we've checked.
+            long newPtr = appendEntry(key, value);
+            frame.putLong(frameOff + 2 * SLOT_SIZE, newPtr);
+        }
+        return null;
+    }
+
+    public long appendEntry(BinaryEntry key, BinaryEntry value) {
+        ByteBuffer frame = frames.get(currFrameIndex);
+        int requiredSpace = key.len + value.len + ENTRY_HEADER_SIZE;
+        if (nextOff + requiredSpace >= frameSize) {
+            // Entry doesn't fit on frame, allocate a new one.
+            if (requiredSpace > frameSize) {
+                throw new IllegalStateException("Key and value greater than framesize.");
+            }
+            frames.add(ByteBuffer.allocate(frameSize));
+            currFrameIndex++;
+            nextOff = 0;
+            frame = frames.get(currFrameIndex);
+        }
+        writeEntryHeader(frame, nextOff, key.len, value.len, NULL_PTR);
+        System.arraycopy(key.buf, key.off, frame.array(), nextOff + ENTRY_HEADER_SIZE, key.len);
+        System.arraycopy(value.buf, value.off, frame.array(), nextOff + ENTRY_HEADER_SIZE + key.len, value.len);
+        long entryPtr = getEntryPtr(currFrameIndex, nextOff);
+        nextOff += requiredSpace;
+        size++;
+        return entryPtr;
+    }
+
+    private void writeEntryHeader(ByteBuffer frame, int targetOff, int keyLen, int valLen, long ptr) {
+        frame.putShort(targetOff, (short) keyLen);
+        frame.putShort(targetOff + SLOT_SIZE, (short) valLen);
+        frame.putLong(targetOff + 2 * SLOT_SIZE, ptr);
+    }
+
+    private long getEntryPtr(int frameIndex, int frameOff) {
         return (((long) frameIndex) << 32) + frameOff;
     }
 
@@ -175,93 +179,94 @@
         return (int) (ptr & 0xffffffff);
     }
 
-	public int size() {
-		return size;
-	}
+    public int size() {
+        return size;
+    }
 
-	public boolean isEmpty() {
-		return size > 0;
-	}
+    public boolean isEmpty() {
+        return size > 0;
+    }
 
-	public void clear() {
-		// Initialize all entries to point to nothing.
-		Arrays.fill(listHeads, NULL_PTR);
-		currFrameIndex = 0;
-		nextOff = 0;
-		size = 0;
-	}
-	
-	public Iterator<Pair<BinaryEntry, BinaryEntry>> iterator() {
-		return new BinaryHashMapIterator();
-	}
-	
-	public class BinaryHashMapIterator implements Iterator<Pair<BinaryEntry, BinaryEntry> > {
-		private final Pair<BinaryEntry, BinaryEntry> val = new Pair<BinaryEntry, BinaryEntry>(new BinaryEntry(), new BinaryEntry());
-		private int listHeadIndex;
-		private ByteBuffer frame;
-		private int frameIndex;
-		private int frameOff;
-		
-		public BinaryHashMapIterator() {
-			listHeadIndex = 0;
-			frame = null;
-			frameIndex = -1;
-			frameOff = -1;
-		}
-		
-		@Override
-		public boolean hasNext() {
-			if (frame != null) {
-				long nextPtr = frame.getLong(frameOff + 2 * SLOT_SIZE);
-				if (nextPtr == NULL_PTR) {
-					// End of current list.
-					listHeadIndex++;
-					return nextListHead();
-				} else {
-					// Follow pointer.
-					setValue(nextPtr);
-					return true;
-				}
-			}
-			return nextListHead();
-		}
+    public void clear() {
+        // Initialize all entries to point to nothing.
+        Arrays.fill(listHeads, NULL_PTR);
+        currFrameIndex = 0;
+        nextOff = 0;
+        size = 0;
+    }
 
-		private boolean nextListHead() {
-			// Position to first non-null list-head pointer.
-			while(listHeadIndex < listHeads.length && listHeads[listHeadIndex] == NULL_PTR) {
-				listHeadIndex++;
-			}
-			if (listHeadIndex < listHeads.length) {
-				// Positioned to first non-null list head.
-				setValue(listHeads[listHeadIndex]);
-				return true;
-			} else {
-				// No more lists.
-				frame = null;
-				return false;
-			}
-		}
-		
-		private void setValue(long ptr) {
-			frameIndex = getFrameIndex(ptr);
-			frameOff = getFrameOffset(ptr);
-			frame = frames.get(frameIndex);
-			int entryKeyOff = frameOff + ENTRY_HEADER_SIZE;
-			int entryKeyLen = frame.getShort(frameOff);
-			int entryValOff = frameOff + ENTRY_HEADER_SIZE + entryKeyLen;
-			int entryValLen = frame.getShort(frameOff + SLOT_SIZE);
-			val.first.set(frame.array(), entryKeyOff, entryKeyLen);
-			val.second.set(frame.array(), entryValOff, entryValLen);
-		}
-		
-		@Override
-		public Pair<BinaryEntry, BinaryEntry> next() {
-			return val;
-		}
+    public Iterator<Pair<BinaryEntry, BinaryEntry>> iterator() {
+        return new BinaryHashMapIterator();
+    }
 
-		@Override
-		public void remove() {
-			throw new UnsupportedOperationException("Remove not implemented");
-		}
-	}
+    public class BinaryHashMapIterator implements Iterator<Pair<BinaryEntry, BinaryEntry>> {
+        private final Pair<BinaryEntry, BinaryEntry> val = new Pair<BinaryEntry, BinaryEntry>(new BinaryEntry(),
+                new BinaryEntry());
+        private int listHeadIndex;
+        private ByteBuffer frame;
+        private int frameIndex;
+        private int frameOff;
+
+        public BinaryHashMapIterator() {
+            listHeadIndex = 0;
+            frame = null;
+            frameIndex = -1;
+            frameOff = -1;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (frame != null) {
+                long nextPtr = frame.getLong(frameOff + 2 * SLOT_SIZE);
+                if (nextPtr == NULL_PTR) {
+                    // End of current list.
+                    listHeadIndex++;
+                    return nextListHead();
+                } else {
+                    // Follow pointer.
+                    setValue(nextPtr);
+                    return true;
+                }
+            }
+            return nextListHead();
+        }
+
+        private boolean nextListHead() {
+            // Position to first non-null list-head pointer.
+            while (listHeadIndex < listHeads.length && listHeads[listHeadIndex] == NULL_PTR) {
+                listHeadIndex++;
+            }
+            if (listHeadIndex < listHeads.length) {
+                // Positioned to first non-null list head.
+                setValue(listHeads[listHeadIndex]);
+                return true;
+            } else {
+                // No more lists.
+                frame = null;
+                return false;
+            }
+        }
+
+        private void setValue(long ptr) {
+            frameIndex = getFrameIndex(ptr);
+            frameOff = getFrameOffset(ptr);
+            frame = frames.get(frameIndex);
+            int entryKeyOff = frameOff + ENTRY_HEADER_SIZE;
+            int entryKeyLen = frame.getShort(frameOff);
+            int entryValOff = frameOff + ENTRY_HEADER_SIZE + entryKeyLen;
+            int entryValLen = frame.getShort(frameOff + SLOT_SIZE);
+            val.first.set(frame.array(), entryKeyOff, entryKeyLen);
+            val.second.set(frame.array(), entryValOff, entryValLen);
+        }
+
+        @Override
+        public Pair<BinaryEntry, BinaryEntry> next() {
+            return val;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException("Remove not implemented");
+        }
+    }
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
index 1a3e1b8..14651fe 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
@@ -21,9 +21,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
 
-/**
- * @author Xiaoyu Ma
- */
 public class CodePointToStringDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodepointIterator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodepointIterator.java
index f103e0f..1365277 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodepointIterator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CodepointIterator.java
@@ -2,10 +2,6 @@
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
-/**
- *
- * @author Xiaoyu Ma
- */
 public class CodepointIterator {
     public void reset(byte [] buf, int startPos) {
         this.buf = buf;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
index 0181698..10aea29 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
@@ -27,7 +27,7 @@
             if (y > 0 && z < 0)
                 throw new ArithmeticException("Overflow adding " + x + " + " + y);
         } else if (y < 0 && z > 0)
-            throw new ArithmeticException("Overflow adding " + x + " + " + y);
+            throw new ArithmeticException("Underflow adding " + x + " + " + y);
         return z;
     }
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
new file mode 100644
index 0000000..208c454
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NumericCaretDescriptor extends AbstractNumericArithmeticEval {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericCaretDescriptor();
+        }
+    };
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateInteger(long, long)
+     */
+    @Override
+    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
+        double result = Math.pow(lhs, rhs);
+        if (result > Long.MAX_VALUE) {
+            throw new ArithmeticException("Overflow of caret operation: " + lhs + " ^ " + rhs);
+        }
+        if (result < Long.MIN_VALUE) {
+            throw new ArithmeticException("Underflow of caret operation: " + lhs + " ^ " + rhs);
+        }
+        return (long) result;
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateDouble(double, double)
+     */
+    @Override
+    protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
+        return Math.pow(lhs, rhs);
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+     */
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.CARET;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringConcatDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
index bcd199c..5ffcb29 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
@@ -25,9 +25,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringConcatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java
index 6d0eaad..3e08fa3 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java
@@ -18,9 +18,6 @@
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringEndWithDescrtiptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEqualDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
index e921e45..e778f3e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
@@ -14,9 +14,6 @@
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringEqualDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
new file mode 100644
index 0000000..98a70b4
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.util.regex.Pattern;
+
+import edu.uci.ics.asterix.om.base.AString;
+
+public class StringEvaluatorUtils {
+
+    public static int toFlag(AString pattern) {
+        String str = pattern.getStringValue();
+        int flag = 0;
+        for (int i = 0; i < str.length(); i++) {
+            char c = str.charAt(i);
+            switch (c) {
+                case 's':
+                    flag |= Pattern.DOTALL;
+                    break;
+                case 'm':
+                    flag |= Pattern.MULTILINE;
+                    break;
+                case 'i':
+                    flag |= Pattern.CASE_INSENSITIVE;
+                    break;
+                case 'x':
+                    flag |= Pattern.COMMENTS;
+                    break;
+            }
+        }
+        return flag;
+    }
+
+    public final static char[] reservedRegexChars = new char[] { '$', '(', ')', '*', '.', '[', '\\', ']', '^', '{',
+            '|', '}' };
+    
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringJoinDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
index 9c2f960..7742ccc 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
@@ -21,9 +21,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringJoinDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
index 5e188d3..d8d0b55 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
@@ -23,9 +23,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringLowerCaseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java
index 03a3545..e26a0c6 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java
@@ -26,9 +26,6 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringMatchesDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java
index 93dc342..6da320b 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java
@@ -30,9 +30,6 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringMatchesWithFlagDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
@@ -112,7 +109,7 @@
                                 DataInputStream di = new DataInputStream(new ByteArrayInputStream(
                                         lastFlags.getByteArray()));
                                 astrFlags = (AString) stringSerde.deserialize(di);
-                                flags = toFlag(astrFlags);
+                                flags = StringEvaluatorUtils.toFlag(astrFlags);
                             }
 
                             pattern = Pattern.compile(strPattern, flags);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
index 230eb34..c8917dc 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
@@ -26,9 +26,6 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringReplaceDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceWithFlagsDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceWithFlagsDescriptor.java
index 4f18863..74c30e7 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceWithFlagsDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringReplaceWithFlagsDescriptor.java
@@ -26,9 +26,6 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringReplaceWithFlagsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
@@ -130,7 +127,7 @@
                                 DataInputStream di = new DataInputStream(new ByteArrayInputStream(
                                         lastFlags.getByteArray()));
                                 astrFlags = (AString) stringSerde.deserialize(di);
-                                flags = toFlag(astrFlags);
+                                flags = StringEvaluatorUtils.toFlag(astrFlags);
                             }
 
                             if (newPattern || newFlags)
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringStartWithDescrtiptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringStartWithDescrtiptor.java
index 601c3d0..c852fbc 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringStartWithDescrtiptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringStartWithDescrtiptor.java
@@ -14,9 +14,6 @@
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringStartWithDescrtiptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
index e001d9e..d34280e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
@@ -25,18 +25,10 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-/**
- * @author Xiaoyu Ma
- */
 public class StringToCodePointDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
 
-    public static final IFunctionDescriptorFactory FACTORY1 = new IFunctionDescriptorFactory() {
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new StringToCodePointDescriptor();
-        }
-    };
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new StringToCodePointDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
index 77911af..61d5d72 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
@@ -72,8 +72,7 @@
                         int posPattern = 3;
 
                         int offset = 0;
-                        while (posSrc - 3 < srcLen - patternLen) {
-                            // offset = 0;                        
+                        while (posSrc - 3 < srcLen - patternLen) {                       
                             while (posPattern + offset - 3 < patternLen && posSrc + offset - 3 < srcLen) {
                                 char c1 = UTF8StringPointable.charAt(src, posSrc + offset);
                                 char c2 = UTF8StringPointable.charAt(pattern, posPattern + offset);
@@ -82,7 +81,6 @@
                                 offset++;
                             }
                             if (offset == patternLen) {
-                                // found = true;
                                 break;
                             }
                             posSrc += UTF8StringPointable.charSize(src, posSrc);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
index 441d20a..55b9663 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
@@ -120,8 +120,8 @@
 
                             StringBuilder sbder = new StringBuilder();
 
-                            calInstance.getExtendStringRepWithTimezoneUntilField(chronon, timezone, sbder, Fields.YEAR,
-                                    Fields.MILLISECOND);
+                            calInstance.getExtendStringRepUntilField(chronon, timezone, sbder, Fields.YEAR,
+                                    Fields.MILLISECOND, true);
 
                             out.writeByte(SER_STRING_TYPE_TAG);
                             out.writeUTF(sbder.toString());
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
index 1ca64c6..340545d 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
@@ -120,8 +120,8 @@
 
                             StringBuilder sbder = new StringBuilder();
 
-                            calInstance.getExtendStringRepWithTimezoneUntilField(chronon, timezone, sbder, Fields.HOUR,
-                                    Fields.MILLISECOND);
+                            calInstance.getExtendStringRepUntilField(chronon, timezone, sbder, Fields.HOUR,
+                                    Fields.MILLISECOND, true);
 
                             out.writeByte(SER_STRING_TYPE_TAG);
                             out.writeUTF(sbder.toString());
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
index c6da17e..b343f63 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
@@ -187,8 +187,9 @@
 
                                 if (day < 0) {
                                     boolean isLeapYear = calInstanct.isLeapYear(year1);
-                                    day += (isLeapYear) ? (GregorianCalendarSystem.DAYS_OF_MONTH_LEAP[month1 - 2])
-                                            : (GregorianCalendarSystem.DAYS_OF_MONTH_ORDI[month1 - 2]);
+                                    // need to "borrow" the days in previous month to make the day positive; when month is 1 (Jan), Dec will be borrowed
+                                    day += (isLeapYear) ? (GregorianCalendarSystem.DAYS_OF_MONTH_LEAP[(12 + month1 - 2) % 12])
+                                            : (GregorianCalendarSystem.DAYS_OF_MONTH_ORDI[(12 + month1 - 2) % 12]);
                                     month -= 1;
                                 }
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
index e09f1da..a934473 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
@@ -201,8 +201,9 @@
 
                                 if (day < 0) {
                                     boolean isLeapYear = calInstanct.isLeapYear(year1);
-                                    day += (isLeapYear) ? (GregorianCalendarSystem.DAYS_OF_MONTH_LEAP[month1 - 2])
-                                            : (GregorianCalendarSystem.DAYS_OF_MONTH_ORDI[month1 - 2]);
+                                    // need to "borrow" the days in previous month to make the day positive; when month is 1 (Jan), Dec will be borrowed
+                                    day += (isLeapYear) ? (GregorianCalendarSystem.DAYS_OF_MONTH_LEAP[(12 + month1 - 2) % 12])
+                                            : (GregorianCalendarSystem.DAYS_OF_MONTH_ORDI[(12 + month1 - 2) % 12]);
                                     month -= 1;
                                 }
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
index 2c81b7c..a5cb0f1 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -21,6 +21,7 @@
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryIntegerInspector;
 import edu.uci.ics.asterix.formats.nontagged.AqlJSONPrinterFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlNormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlPredicateEvaluatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlPrinterFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
@@ -149,6 +150,7 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.NotNullDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NumericAbsDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NumericAddDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.NumericCaretDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NumericCeilingDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NumericDivideDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NumericFloorDescriptor;
@@ -273,6 +275,7 @@
 import edu.uci.ics.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.evaluators.ConstantEvalFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
@@ -347,6 +350,7 @@
         temp.add(NumericMultiplyDescriptor.FACTORY);
         temp.add(NumericSubtractDescriptor.FACTORY);
         temp.add(NumericModuloDescriptor.FACTORY);
+        temp.add(NumericCaretDescriptor.FACTORY);
         temp.add(IsNullDescriptor.FACTORY);
         temp.add(NotDescriptor.FACTORY);
         temp.add(LenDescriptor.FACTORY);
@@ -935,4 +939,9 @@
         return AqlBinaryHashFunctionFamilyProvider.INSTANCE;
     }
 
+	@Override
+	public IPredicateEvaluatorFactoryProvider getPredicateEvaluatorFactoryProvider() {
+		return AqlPredicateEvaluatorFactoryProvider.INSTANCE;
+	}
+
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
index 45e19c6..f0e8ec5 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
@@ -1,12 +1,12 @@
 package edu.uci.ics.asterix.runtime.job.listener;
 
-import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext.TransactionType;
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
 import edu.uci.ics.hyracks.api.job.IJobletEventListener;
 import edu.uci.ics.hyracks.api.job.IJobletEventListenerFactory;
@@ -22,7 +22,7 @@
         this.jobId = jobId;
         this.transactionalWrite = transactionalWrite;
     }
-    
+
     public JobId getJobId() {
         return jobId;
     }
@@ -34,12 +34,13 @@
             @Override
             public void jobletFinish(JobStatus jobStatus) {
                 try {
-                    ITransactionManager txnManager = ((AsterixAppRuntimeContext) jobletContext.getApplicationContext()
+                    ITransactionManager txnManager = ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext()
                             .getApplicationObject()).getTransactionSubsystem().getTransactionManager();
-                    TransactionContext txnContext = txnManager.getTransactionContext(jobId);
+                    ITransactionContext txnContext = txnManager.getTransactionContext(jobId);
                     txnContext.setTransactionType(transactionalWrite ? TransactionType.READ_WRITE
                             : TransactionType.READ);
-                    txnManager.completedTransaction(txnContext, new DatasetId(-1), -1, !(jobStatus == JobStatus.FAILURE));
+                    txnManager.completedTransaction(txnContext, new DatasetId(-1), -1,
+                            !(jobStatus == JobStatus.FAILURE));
                 } catch (ACIDException e) {
                     throw new Error(e);
                 }
@@ -48,7 +49,7 @@
             @Override
             public void jobletStart() {
                 try {
-                    ((AsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
+                    ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
                             .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId);
                 } catch (ACIDException e) {
                     throw new Error(e);
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
index 3d28851..6b177a7 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
@@ -36,6 +36,7 @@
 import edu.uci.ics.asterix.common.annotations.TypeDataGen;
 import edu.uci.ics.asterix.common.annotations.UndeclaredFieldsDataGen;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.om.types.ARecordType;
@@ -46,8 +47,7 @@
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.types.TypeSignature;
 import edu.uci.ics.asterix.tools.translator.ADGenDmlTranslator;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.data.utils.WriteValueTools;
@@ -1000,4 +1000,4 @@
         }
     }
 
-}
+}
\ No newline at end of file
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
index 84b989d..afdf343 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
@@ -19,7 +19,6 @@
 import java.util.Map;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
 import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.ITypedDatasourceAdapter;
 import edu.uci.ics.asterix.feed.managed.adapter.IManagedFeedAdapter;
@@ -47,48 +46,15 @@
         IManagedFeedAdapter {
 
     private static final long serialVersionUID = 1L;
+    private FileSystemBasedAdapter coreAdapter;
+    private String format;
 
-    public static final String KEY_FILE_SYSTEM = "fs";
-    public static final String LOCAL_FS = "localfs";
-    public static final String HDFS = "hdfs";
-
-    private final FileSystemBasedAdapter coreAdapter;
-    private final Map<String, String> configuration;
-    private final String fileSystem;
-    private final String format;
-
-    public RateControlledFileSystemBasedAdapter(ARecordType atype, Map<String, String> configuration) throws Exception {
+    public RateControlledFileSystemBasedAdapter(ARecordType atype, Map<String, Object> configuration,
+            FileSystemBasedAdapter coreAdapter, String format) throws Exception {
         super(atype);
-        checkRequiredArgs(configuration);
-        fileSystem = configuration.get(KEY_FILE_SYSTEM);
-        String adapterFactoryClass = null;
-        if (fileSystem.equalsIgnoreCase(LOCAL_FS)) {
-            adapterFactoryClass = "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory";
-        } else if (fileSystem.equals(HDFS)) {
-            adapterFactoryClass = "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory";
-        } else {
-            throw new AsterixException("Unsupported file system type " + fileSystem);
-        }
-        format = configuration.get(KEY_FORMAT);
-        IGenericDatasetAdapterFactory adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(
-                adapterFactoryClass).newInstance();
-        coreAdapter = (FileSystemBasedAdapter) adapterFactory.createAdapter(configuration, atype);
         this.configuration = configuration;
-    }
-
-    private void checkRequiredArgs(Map<String, String> configuration) throws Exception {
-        if (configuration.get(KEY_FILE_SYSTEM) == null) {
-            throw new Exception("File system type not specified. (fs=?) File system could be 'localfs' or 'hdfs'");
-        }
-        if (configuration.get(IGenericDatasetAdapterFactory.KEY_TYPE_NAME) == null) {
-            throw new Exception("Record type not specified (output-type-name=?)");
-        }
-        if (configuration.get(KEY_PATH) == null) {
-            throw new Exception("File path not specified (path=?)");
-        }
-        if (configuration.get(KEY_FORMAT) == null) {
-            throw new Exception("File format not specified (format=?)");
-        }
+        this.coreAdapter = coreAdapter;
+        this.format = format;
     }
 
     @Override
@@ -103,7 +69,7 @@
     }
 
     @Override
-    public void configure(Map<String, String> arguments) throws Exception {
+    public void configure(Map<String, Object> arguments) throws Exception {
         coreAdapter.configure(arguments);
     }
 
@@ -189,16 +155,16 @@
 
     private final ARecordType recordType;
     private final IDataParser dataParser;
-    private final Map<String, String> configuration;
+    private final Map<String, Object> configuration;
 
     public RateControlledTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
-            char fieldDelimiter, Map<String, String> configuration) {
+            char fieldDelimiter, Map<String, Object> configuration) {
         this.recordType = recordType;
         dataParser = new DelimitedDataParser(recordType, valueParserFactories, fieldDelimiter);
         this.configuration = configuration;
     }
 
-    public RateControlledTupleParserFactory(ARecordType recordType, Map<String, String> configuration) {
+    public RateControlledTupleParserFactory(ARecordType recordType, Map<String, Object> configuration) {
         this.recordType = recordType;
         dataParser = new ADMDataParser();
         this.configuration = configuration;
@@ -221,10 +187,10 @@
     public static final String INTER_TUPLE_INTERVAL = "tuple-interval";
 
     public RateControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser dataParser,
-            Map<String, String> configuration) {
+            Map<String, Object> configuration) {
         super(ctx, recType);
         this.dataParser = dataParser;
-        String propValue = configuration.get(INTER_TUPLE_INTERVAL);
+        String propValue = (String) configuration.get(INTER_TUPLE_INTERVAL);
         if (propValue != null) {
             interTupleInterval = Long.parseLong(propValue);
         } else {
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
index 6c32acb..bf1c268 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
@@ -16,7 +16,9 @@
 
 import java.util.Map;
 
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
+import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
@@ -28,10 +30,37 @@
  * source file has been ingested.
  */
 public class RateControlledFileSystemBasedAdapterFactory implements IGenericDatasetAdapterFactory {
+    private static final long serialVersionUID = 1L;
+    
+    public static final String KEY_FILE_SYSTEM = "fs";
+    public static final String LOCAL_FS = "localfs";
+    public static final String HDFS = "hdfs";
+    public static final String KEY_PATH = "path";
+    public static final String KEY_FORMAT = "format";
+
+    private IGenericDatasetAdapterFactory adapterFactory;
+    private String format;
+    private boolean setup = false;
 
     @Override
-    public IDatasourceAdapter createAdapter(Map<String, String> configuration, IAType type) throws Exception {
-        return new RateControlledFileSystemBasedAdapter((ARecordType) type, configuration);
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType type) throws Exception {
+        if (!setup) {
+            checkRequiredArgs(configuration);
+            String fileSystem = (String) configuration.get(KEY_FILE_SYSTEM);
+            String adapterFactoryClass = null;
+            if (fileSystem.equalsIgnoreCase(LOCAL_FS)) {
+                adapterFactoryClass = "edu.uci.ics.asterix.external.adapter.factory.NCFileSystemAdapterFactory";
+            } else if (fileSystem.equals(HDFS)) {
+                adapterFactoryClass = "edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory";
+            } else {
+                throw new AsterixException("Unsupported file system type " + fileSystem);
+            }
+            format = (String) configuration.get(KEY_FORMAT);
+            adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClass).newInstance();
+            setup = true;
+        }
+        return new RateControlledFileSystemBasedAdapter((ARecordType) type, configuration,
+                (FileSystemBasedAdapter) adapterFactory.createAdapter(configuration, type), format);
     }
 
     @Override
@@ -39,4 +68,19 @@
         return "file_feed";
     }
 
+    private void checkRequiredArgs(Map<String, Object> configuration) throws Exception {
+        if (configuration.get(KEY_FILE_SYSTEM) == null) {
+            throw new Exception("File system type not specified. (fs=?) File system could be 'localfs' or 'hdfs'");
+        }
+        if (configuration.get(IGenericDatasetAdapterFactory.KEY_TYPE_NAME) == null) {
+            throw new Exception("Record type not specified (output-type-name=?)");
+        }
+        if (configuration.get(KEY_PATH) == null) {
+            throw new Exception("File path not specified (path=?)");
+        }
+        if (configuration.get(KEY_FORMAT) == null) {
+            throw new Exception("File format not specified (format=?)");
+        }
+    }
+
 }
\ No newline at end of file
diff --git a/asterix-transactions/pom.xml b/asterix-transactions/pom.xml
index 7d951ce..d09ce8a 100644
--- a/asterix-transactions/pom.xml
+++ b/asterix-transactions/pom.xml
@@ -40,6 +40,13 @@
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
 		</dependency>
+                <dependency>
+                        <groupId>edu.uci.ics.asterix</groupId>
+                        <artifactId>asterix-common</artifactId>
+                        <version>0.0.6-SNAPSHOT</version>
+                        <type>jar</type>
+                        <scope>compile</scope>
+                </dependency>
 	</dependencies>
 
 </project>
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java
deleted file mode 100644
index 2016d08..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Copyright 2009-2012 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.transaction.management.opcallbacks;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
-import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
-
-public class IndexOperationTracker implements ILSMOperationTracker {
-
-    // Number of active operations on a ILSMIndex instance.
-    private AtomicInteger numActiveOperations;
-    private long lastLSN;
-    private long firstLSN;
-    private final ILSMIndex index;
-    private final ILSMIOOperationCallback ioOpCallback;
-    private ILSMIndexAccessor accessor;
-
-    public IndexOperationTracker(ILSMIndex index, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
-        this.numActiveOperations = new AtomicInteger(0);
-        this.index = index;
-        //TODO 
-        //This code is added to avoid NullPointException when the index's comparatorFactory is null.
-        //The null comparator factory is set in the constructor of the IndexDropOperatorDescriptor.
-        if (ioOpCallbackFactory != null) {
-            ioOpCallback = ioOpCallbackFactory.createIOOperationCallback(this);
-        } else {
-            ioOpCallback = NoOpIOOperationCallback.INSTANCE;
-        }
-        resetLSNs();
-    }
-
-    @Override
-    public void beforeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
-            IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        if (opType != LSMOperationType.FORCE_MODIFICATION) {
-            numActiveOperations.incrementAndGet();
-
-            // Increment transactor-local active operations count.
-            AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
-            if (opCallback != null) {
-                opCallback.incrementLocalNumActiveOperations();
-            }
-        }
-    }
-
-    @Override
-    public void afterOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
-            IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        // Searches are immediately considered complete, because they should not prevent the execution of flushes.
-        if (searchCallback != null) {
-            completeOperation(opType, searchCallback, modificationCallback);
-        }
-    }
-
-    @Override
-    public void completeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
-            IModificationOperationCallback modificationCallback) throws HyracksDataException {
-
-        // Decrement transactor-local active operations count.
-        AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
-        if (opCallback != null) {
-            opCallback.decrementLocalNumActiveOperations();
-        }
-        // If we need a flush, and this is the last completing operation, then schedule the flush.
-        // Once the flush has completed notify all waiting operations.
-        if (index.getFlushStatus() && numActiveOperations.decrementAndGet() == 0 && opType != LSMOperationType.FLUSH) {
-            if (accessor == null) {
-                accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
-                        NoOpOperationCallback.INSTANCE);
-            }
-            accessor.scheduleFlush(ioOpCallback);
-        }
-    }
-
-    private AbstractOperationCallback getOperationCallback(ISearchOperationCallback searchCallback,
-            IModificationOperationCallback modificationCallback) {
-
-        if (searchCallback == NoOpOperationCallback.INSTANCE || modificationCallback == NoOpOperationCallback.INSTANCE) {
-            return null;
-        }
-        if (searchCallback != null) {
-            return (AbstractOperationCallback) searchCallback;
-        } else {
-            return (AbstractOperationCallback) modificationCallback;
-        }
-    }
-
-    public ILSMIOOperationCallback getIOOperationCallback() {
-        return ioOpCallback;
-    }
-
-    public long getLastLSN() {
-        return lastLSN;
-    }
-
-    public long getFirstLSN() {
-        return firstLSN;
-    }
-
-    public void updateLastLSN(long lastLSN) {
-        if (firstLSN == -1) {
-            firstLSN = lastLSN;
-        }
-        this.lastLSN = Math.max(this.lastLSN, lastLSN);
-    }
-
-    public void resetLSNs() {
-        lastLSN = -1;
-        firstLSN = -1;
-    }
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTrackerFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTrackerFactory.java
deleted file mode 100644
index 032a4f9..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTrackerFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright 2009-2012 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.transaction.management.opcallbacks;
-
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
-
-public class IndexOperationTrackerFactory implements ILSMOperationTrackerFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
-    
-    public IndexOperationTrackerFactory(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
-        this.ioOpCallbackFactory = ioOpCallbackFactory;
-    }
-    
-    @Override
-    public ILSMOperationTracker createOperationTracker(ILSMIndex index) {
-        return new IndexOperationTracker(index, ioOpCallbackFactory);
-    }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallback.java
index a78f2ff..dedc5df 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallback.java
@@ -15,9 +15,10 @@
 
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -30,7 +31,7 @@
         ISearchOperationCallback {
 
     public PrimaryIndexInstantSearchOperationCallback(int datasetId, int[] entityIdFields, ILockManager lockManager,
-            TransactionContext txnCtx) {
+            ITransactionContext txnCtx) {
         super(datasetId, entityIdFields, txnCtx, lockManager);
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
index 3773950..3efd90b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
@@ -15,11 +15,12 @@
 
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
@@ -38,9 +39,9 @@
     @Override
     public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
             throws HyracksDataException {
-        TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+        ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
-            TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
             return new PrimaryIndexInstantSearchOperationCallback(datasetId, primaryKeyFields,
                     txnSubsystem.getLockManager(), txnCtx);
         } catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
index 824a324..9097570 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
@@ -15,12 +15,14 @@
 
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -37,11 +39,11 @@
     protected final long resourceId;
     protected final byte resourceType;
     protected final IndexOperation indexOp;
-    protected final TransactionSubsystem txnSubsystem;
+    protected final ITransactionSubsystem txnSubsystem;
 
-    public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
-            TransactionContext txnCtx, ILockManager lockManager,
-            TransactionSubsystem txnSubsystem, long resourceId, byte resourceType, IndexOperation indexOp) {
+    public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
+            ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType,
+            IndexOperation indexOp) {
         super(datasetId, primaryKeyFields, txnCtx, lockManager);
         this.resourceId = resourceId;
         this.resourceType = resourceType;
@@ -61,19 +63,19 @@
 
     @Override
     public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
-        IndexLogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
-        int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
-        LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
-        IndexOperation oldOp = IndexOperation.INSERT;
-        if (before == null) {
-            oldOp = IndexOperation.NOOP;
-        }
-        if (lsmBTreeTuple != null && lsmBTreeTuple.isAntimatter()) {
-            oldOp = IndexOperation.DELETE;
-        }
         try {
-            logger.generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId, indexOp, after,
-                    oldOp, before);
+            ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
+            int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
+            LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
+            IndexOperation oldOp = IndexOperation.INSERT;
+            if (before == null) {
+                oldOp = IndexOperation.NOOP;
+            }
+            if (lsmBTreeTuple != null && lsmBTreeTuple.isAntimatter()) {
+                oldOp = IndexOperation.DELETE;
+            }
+            ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
+                    indexOp, after, oldOp, before);
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index c75ab6f..038ecee 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -15,11 +15,13 @@
 
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
@@ -47,7 +49,7 @@
     public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
             IHyracksTaskContext ctx) throws HyracksDataException {
 
-        TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+        ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
         ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
@@ -56,7 +58,7 @@
         }
 
         try {
-            TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
             IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
                     primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
                     indexOp);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerProvider.java
new file mode 100644
index 0000000..74be361
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerProvider.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.opcallbacks;
+
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+
+public class PrimaryIndexOperationTrackerProvider implements ILSMOperationTrackerProvider {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int datasetID;
+
+    public PrimaryIndexOperationTrackerProvider(int datasetID) {
+        this.datasetID = datasetID;
+    }
+
+    @Override
+    public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
+        DatasetLifecycleManager dslcManager = (DatasetLifecycleManager) ((IAsterixAppRuntimeContext) ctx
+                .getJobletContext().getApplicationContext().getApplicationObject()).getIndexLifecycleManager();
+        return dslcManager.getOperationTracker(datasetID);
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallback.java
index 62ec3c9..eee33e9 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallback.java
@@ -15,9 +15,10 @@
 
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -29,7 +30,7 @@
 public class PrimaryIndexSearchOperationCallback extends AbstractOperationCallback implements ISearchOperationCallback {
 
     public PrimaryIndexSearchOperationCallback(int datasetId, int[] entityIdFields, ILockManager lockManager,
-            TransactionContext txnCtx) {
+            ITransactionContext txnCtx) {
         super(datasetId, entityIdFields, txnCtx, lockManager);
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
index fc62b90..5d837b0 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
@@ -15,11 +15,12 @@
 
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
@@ -38,9 +39,9 @@
     @Override
     public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
             throws HyracksDataException {
-        TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+        ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
-            TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
             return new PrimaryIndexSearchOperationCallback(datasetId, primaryKeyFields, txnSubsystem.getLockManager(),
                     txnCtx);
         } catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
index 5b55e9a..2287ed7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
@@ -15,12 +15,14 @@
 
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -39,10 +41,10 @@
     protected final byte resourceType;
     protected final IndexOperation indexOp;
     protected final IndexOperation oldOp;
-    protected final TransactionSubsystem txnSubsystem;
+    protected final ITransactionSubsystem txnSubsystem;
 
     public SecondaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
-            TransactionContext txnCtx, ILockManager lockManager, TransactionSubsystem txnSubsystem, long resourceId,
+            ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
             byte resourceType, IndexOperation indexOp) {
         super(datasetId, primaryKeyFields, txnCtx, lockManager);
         this.resourceId = resourceId;
@@ -59,9 +61,10 @@
 
     @Override
     public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
-        IndexLogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
-        int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
+
         try {
+            ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
+            int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
             IndexOperation effectiveOldOp;
             if (resourceType == ResourceType.LSM_BTREE) {
                 LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
@@ -75,8 +78,8 @@
             } else {
                 effectiveOldOp = oldOp;
             }
-            logger.generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId, indexOp, after,
-                    effectiveOldOp, before);
+            ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
+                    indexOp, after, effectiveOldOp, before);
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index 672b434..68a09de 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -15,11 +15,12 @@
 
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
@@ -44,7 +45,7 @@
     public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
             IHyracksTaskContext ctx) throws HyracksDataException {
 
-        TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+        ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
         ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
@@ -53,7 +54,7 @@
         }
 
         try {
-            TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
             return new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
                     txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
         } catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
new file mode 100644
index 0000000..4024836
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
@@ -0,0 +1,24 @@
+package edu.uci.ics.asterix.transaction.management.opcallbacks;
+
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+
+public class SecondaryIndexOperationTrackerProvider implements ILSMOperationTrackerProvider {
+
+    private static final long serialVersionUID = 1L;
+
+    private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+
+    public SecondaryIndexOperationTrackerProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
+    }
+
+    @Override
+    public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
+        return new BaseOperationTracker(ioOpCallbackFactory);
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallback.java
index 4c8a583..e6ce589 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallback.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/AbstractLSMLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/AbstractLSMLocalResourceMetadata.java
new file mode 100644
index 0000000..a95b677
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/AbstractLSMLocalResourceMetadata.java
@@ -0,0 +1,18 @@
+package edu.uci.ics.asterix.transaction.management.resource;
+
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
+
+public abstract class AbstractLSMLocalResourceMetadata implements ILocalResourceMetadata {
+
+    private static final long serialVersionUID = 1L;
+
+    protected final int datasetID;
+
+    public AbstractLSMLocalResourceMetadata(int datasetID) {
+        this.datasetID = datasetID;
+    }
+
+    public int getDatasetID() {
+        return datasetID;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
index 7ab0d10..ac4ca61 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
@@ -2,56 +2,62 @@
 
 import java.io.File;
 
-import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
-import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 
-public class LSMBTreeLocalResourceMetadata implements ILocalResourceMetadata {
+public class LSMBTreeLocalResourceMetadata extends AbstractLSMLocalResourceMetadata {
 
     private static final long serialVersionUID = 1L;
 
     private final ITypeTraits[] typeTraits;
     private final IBinaryComparatorFactory[] cmpFactories;
     private final int[] bloomFilterKeyFields;
-    private final int memPageSize;
-    private final int memNumPages;
+    private final boolean isPrimary;
+    private FileSplit[] fileSplits;
+    private int ioDeviceID;
 
     public LSMBTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
-            int[] bloomFilterKeyFields, boolean isPrimary, int memPageSize, int memNumPages) {
+            int[] bloomFilterKeyFields, boolean isPrimary, FileSplit[] fileSplits, int datasetID) {
+        super(datasetID);
         this.typeTraits = typeTraits;
         this.cmpFactories = cmpFactories;
         this.bloomFilterKeyFields = bloomFilterKeyFields;
-        this.memPageSize = memPageSize;
-        this.memNumPages = memNumPages;
+        this.isPrimary = isPrimary;
+        this.fileSplits = fileSplits;
+    }
+
+    public LSMBTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields, boolean isPrimary, int ioDeviceID, int datasetID) {
+        super(datasetID);
+        this.typeTraits = typeTraits;
+        this.cmpFactories = cmpFactories;
+        this.bloomFilterKeyFields = bloomFilterKeyFields;
+        this.isPrimary = isPrimary;
+        this.ioDeviceID = ioDeviceID;
     }
 
     @Override
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) {
         FileReference file = new FileReference(new File(filePath));
-        IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
-                memNumPages, new TransientFileMapManager());
-        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(memNumPages, metaDataFrameFactory);
-        LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager,
-                runtimeContextProvider.getIOManager(), file, runtimeContextProvider.getBufferCache(),
-                runtimeContextProvider.getFileMapManager(), typeTraits, cmpFactories, bloomFilterKeyFields,
-                runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider.getLSMMergePolicy(),
-                runtimeContextProvider.getLSMBTreeOperationTrackerFactory(),
-                runtimeContextProvider.getLSMIOScheduler(),
-                runtimeContextProvider.getLSMBTreeIOOperationCallbackProvider(), partition);
+        IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
+        LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, runtimeContextProvider.getIOManager(),
+                file, runtimeContextProvider.getBufferCache(), runtimeContextProvider.getFileMapManager(), typeTraits,
+                cmpFactories, bloomFilterKeyFields, runtimeContextProvider.getBloomFilterFalsePositiveRate(),
+                runtimeContextProvider.getLSMMergePolicy(),
+                isPrimary ? runtimeContextProvider.getLSMBTreeOperationTracker(datasetID) : new BaseOperationTracker(
+                        LSMBTreeIOOperationCallbackFactory.INSTANCE), runtimeContextProvider.getLSMIOScheduler(),
+                runtimeContextProvider.getLSMBTreeIOOperationCallbackProvider(), fileSplits == null ? ioDeviceID
+                        : fileSplits[partition].getIODeviceId());
         return lsmBTree;
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
index 3bcb747..5d9ac75 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
@@ -1,22 +1,19 @@
 package edu.uci.ics.asterix.transaction.management.resource;
 
-import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
 
-public class LSMInvertedIndexLocalResourceMetadata implements ILocalResourceMetadata {
+public class LSMInvertedIndexLocalResourceMetadata extends AbstractLSMLocalResourceMetadata {
 
     private static final long serialVersionUID = 1L;
 
@@ -25,54 +22,48 @@
     private final ITypeTraits[] tokenTypeTraits;
     private final IBinaryComparatorFactory[] tokenCmpFactories;
     private final IBinaryTokenizerFactory tokenizerFactory;
-    private final int memPageSize;
-    private final int memNumPages;
     private final boolean isPartitioned;
+    private final FileSplit[] fileSplits;
 
     public LSMInvertedIndexLocalResourceMetadata(ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
-            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory, int memPageSize,
-            int memNumPages, boolean isPartitioned) {
+            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+            boolean isPartitioned, FileSplit[] fileSplits, int datasetID) {
+        super(datasetID);
         this.invListTypeTraits = invListTypeTraits;
         this.invListCmpFactories = invListCmpFactories;
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.tokenizerFactory = tokenizerFactory;
-        this.memPageSize = memPageSize;
-        this.memNumPages = memNumPages;
         this.isPartitioned = isPartitioned;
+        this.fileSplits = fileSplits;
     }
 
     @Override
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) throws HyracksDataException {
-
-        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
-                memNumPages);
-        IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
-                metaDataFrameFactory);
+        IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
         try {
             if (isPartitioned) {
-                return InvertedIndexUtils.createPartitionedLSMInvertedIndex(memBufferCache, memFreePageManager,
-                        runtimeContextProvider.getFileMapManager(), invListTypeTraits, invListCmpFactories,
-                        tokenTypeTraits, tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(),
-                        runtimeContextProvider.getIOManager(), filePath,
-                        runtimeContextProvider.getBloomFilterFalsePositiveRate(),
-                        runtimeContextProvider.getLSMMergePolicy(),
-                        runtimeContextProvider.getLSMInvertedIndexOperationTrackerFactory(),
-                        runtimeContextProvider.getLSMIOScheduler(),
-                        runtimeContextProvider.getLSMInvertedIndexIOOperationCallbackProvider(), partition);
+                return InvertedIndexUtils.createPartitionedLSMInvertedIndex(virtualBufferCache, runtimeContextProvider
+                        .getFileMapManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                        tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(),
+                        runtimeContextProvider.getIOManager(), filePath, runtimeContextProvider
+                                .getBloomFilterFalsePositiveRate(), runtimeContextProvider.getLSMMergePolicy(),
+                        new BaseOperationTracker(LSMInvertedIndexIOOperationCallbackFactory.INSTANCE),
+                        runtimeContextProvider.getLSMIOScheduler(), runtimeContextProvider
+                                .getLSMInvertedIndexIOOperationCallbackProvider(), fileSplits[partition]
+                                .getIODeviceId());
             } else {
-                return InvertedIndexUtils.createLSMInvertedIndex(memBufferCache, memFreePageManager,
-                        runtimeContextProvider.getFileMapManager(), invListTypeTraits, invListCmpFactories,
-                        tokenTypeTraits, tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(),
-                        runtimeContextProvider.getIOManager(), filePath,
-                        runtimeContextProvider.getBloomFilterFalsePositiveRate(),
-                        runtimeContextProvider.getLSMMergePolicy(),
-                        runtimeContextProvider.getLSMInvertedIndexOperationTrackerFactory(),
-                        runtimeContextProvider.getLSMIOScheduler(),
-                        runtimeContextProvider.getLSMInvertedIndexIOOperationCallbackProvider(), partition);
+                return InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCache, runtimeContextProvider
+                        .getFileMapManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                        tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(),
+                        runtimeContextProvider.getIOManager(), filePath, runtimeContextProvider
+                                .getBloomFilterFalsePositiveRate(), runtimeContextProvider.getLSMMergePolicy(),
+                        new BaseOperationTracker(LSMInvertedIndexIOOperationCallbackFactory.INSTANCE),
+                        runtimeContextProvider.getLSMIOScheduler(), runtimeContextProvider
+                                .getLSMInvertedIndexIOOperationCallbackProvider(), fileSplits[partition]
+                                .getIODeviceId());
             }
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
index ff26c54..8ad9225 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
@@ -2,26 +2,23 @@
 
 import java.io.File;
 
-import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
-import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
 
-public class LSMRTreeLocalResourceMetadata implements ILocalResourceMetadata {
+public class LSMRTreeLocalResourceMetadata extends AbstractLSMLocalResourceMetadata {
 
     private static final long serialVersionUID = 1L;
 
@@ -31,42 +28,36 @@
     private final IPrimitiveValueProviderFactory[] valueProviderFactories;
     private final RTreePolicyType rtreePolicyType;
     private final ILinearizeComparatorFactory linearizeCmpFactory;
-    private final int memPageSize;
-    private final int memNumPages;
+    private final FileSplit[] fileSplits;
 
     public LSMRTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
-            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int memPageSize,
-            int memNumPages) {
+            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, FileSplit[] fileSplits,
+            int datasetID) {
+        super(datasetID);
         this.typeTraits = typeTraits;
         this.rtreeCmpFactories = rtreeCmpFactories;
         this.btreeCmpFactories = btreeCmpFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
         this.linearizeCmpFactory = linearizeCmpFactory;
-        this.memPageSize = memPageSize;
-        this.memNumPages = memNumPages;
+        this.fileSplits = fileSplits;
     }
 
     @Override
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) throws HyracksDataException {
         FileReference file = new FileReference(new File(filePath));
-        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
-                memNumPages);
-        IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
-                metaDataFrameFactory);
-
+        IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
         try {
-            return LSMRTreeUtils.createLSMTree(memBufferCache, memFreePageManager,
-                    runtimeContextProvider.getIOManager(), file, runtimeContextProvider.getBufferCache(),
-                    runtimeContextProvider.getFileMapManager(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
-                    valueProviderFactories, rtreePolicyType, runtimeContextProvider.getBloomFilterFalsePositiveRate(),
-                    runtimeContextProvider.getLSMMergePolicy(),
-                    runtimeContextProvider.getLSMRTreeOperationTrackerFactory(),
-                    runtimeContextProvider.getLSMIOScheduler(),
-                    runtimeContextProvider.getLSMRTreeIOOperationCallbackProvider(), linearizeCmpFactory, partition);
+            return LSMRTreeUtils.createLSMTree(virtualBufferCache, runtimeContextProvider.getIOManager(), file,
+                    runtimeContextProvider.getBufferCache(), runtimeContextProvider.getFileMapManager(), typeTraits,
+                    rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+                    runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
+                            .getLSMMergePolicy(),
+                    new BaseOperationTracker(LSMRTreeIOOperationCallbackFactory.INSTANCE), runtimeContextProvider
+                            .getLSMIOScheduler(), runtimeContextProvider.getLSMRTreeIOOperationCallbackProvider(),
+                    linearizeCmpFactory, fileSplits[partition].getIODeviceId());
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java
index ed0f79f..707e02e 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java
@@ -1,5 +1,6 @@
 package edu.uci.ics.asterix.transaction.management.resource;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
 import edu.uci.ics.hyracks.storage.common.file.LocalResource;
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java
index 4157c32..6afb9a9 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java
@@ -1,5 +1,6 @@
 package edu.uci.ics.asterix.transaction.management.resource;
 
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 01dce6c..a8f129a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -28,8 +28,8 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
 import edu.uci.ics.hyracks.storage.common.file.LocalResource;
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
@@ -37,26 +37,34 @@
 public class PersistentLocalResourceRepository implements ILocalResourceRepository {
 
     private static final Logger LOGGER = Logger.getLogger(PersistentLocalResourceRepository.class.getName());
-    private final String mountPoint;
-    private static final String ROOT_METADATA_DIRECTORY = "asterix_root_metadata/";
-    private static final String ROOT_METADATA_FILE_NAME_PREFIX = ".asterix_root_metadata_";
+    private final String[] mountPoints;
+    private static final String ROOT_METADATA_DIRECTORY = "asterix_root_metadata";
+    private static final String ROOT_METADATA_FILE_NAME_PREFIX = ".asterix_root_metadata";
     private static final long ROOT_LOCAL_RESOURCE_ID = -4321;
     private static final String METADATA_FILE_NAME = ".metadata";
     private Map<String, LocalResource> name2ResourceMap = new HashMap<String, LocalResource>();
     private Map<Long, LocalResource> id2ResourceMap = new HashMap<Long, LocalResource>();
-    private String rootMetadataFileName;
-    private String rootDir;
+    private final int numIODevices;
 
-    public PersistentLocalResourceRepository(String mountPoint) throws HyracksDataException {
-        File mountPointDir = new File(mountPoint);
-        if (!mountPointDir.exists()) {
-            throw new HyracksDataException(mountPointDir.getAbsolutePath() + "doesn't exist.");
+    public PersistentLocalResourceRepository(List<IODeviceHandle> devices) throws HyracksDataException {
+        numIODevices = devices.size();
+        this.mountPoints = new String[numIODevices];
+        for (int i = 0; i < numIODevices; i++) {
+            String mountPoint = devices.get(i).getPath().getPath();
+            File mountPointDir = new File(mountPoint);
+            if (!mountPointDir.exists()) {
+                throw new HyracksDataException(mountPointDir.getAbsolutePath() + "doesn't exist.");
+            }
+            if (!mountPoint.endsWith(System.getProperty("file.separator"))) {
+                mountPoints[i] = new String(mountPoint + System.getProperty("file.separator"));
+            } else {
+                mountPoints[i] = new String(mountPoint);
+            }
         }
-        if (!mountPoint.endsWith(System.getProperty("file.separator"))) {
-            this.mountPoint = new String(mountPoint + System.getProperty("file.separator"));
-        } else {
-            this.mountPoint = new String(mountPoint);
-        }
+    }
+
+    private String prepareRootMetaDataFileName(String mountPoint, String nodeId, int ioDeviceId) {
+        return mountPoint + ROOT_METADATA_DIRECTORY + "_" + nodeId + "_" + "iodevice" + ioDeviceId;
     }
 
     public void initialize(String nodeId, String rootDir, boolean isNewUniverse, ResourceIdFactory resourceIdFactory)
@@ -64,53 +72,40 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Initializing local resource repository ... ");
         }
-        LocalResource rootLocalResource = null;
 
-        //#. if the rootMetadataFile doesn't exist, create it and return.
-        rootMetadataFileName = new String(mountPoint + ROOT_METADATA_DIRECTORY + ROOT_METADATA_FILE_NAME_PREFIX
-                + nodeId);
-        File rootMetadataFile = new File(rootMetadataFileName);
         if (isNewUniverse) {
-            File rootMetadataDir = new File(mountPoint + ROOT_METADATA_DIRECTORY);
-            if (!rootMetadataDir.exists()) {
-                rootMetadataDir.mkdir();
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("created the root-metadata-file's directory: " + rootMetadataDir.getAbsolutePath());
+            //#. if the rootMetadataFile doesn't exist, create it and return.
+            for (int i = 0; i < numIODevices; i++) {
+                String rootMetadataFileName = prepareRootMetaDataFileName(mountPoints[i], nodeId, i) + File.separator
+                        + ROOT_METADATA_FILE_NAME_PREFIX;
+                File rootMetadataFile = new File(rootMetadataFileName);
+
+                File rootMetadataDir = new File(prepareRootMetaDataFileName(mountPoints[i], nodeId, i));
+                if (!rootMetadataDir.exists()) {
+                    rootMetadataDir.mkdir();
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("created the root-metadata-file's directory: " + rootMetadataDir.getAbsolutePath());
+                    }
                 }
-            }
 
-            rootMetadataFile.delete();
-            if (rootDir.startsWith(System.getProperty("file.separator"))) {
-                this.rootDir = new String(mountPoint + rootDir.substring(System.getProperty("file.separator").length()));
-            } else {
-                this.rootDir = new String(mountPoint + rootDir);
-            }
-            rootLocalResource = new LocalResource(ROOT_LOCAL_RESOURCE_ID, rootMetadataFileName, 0, 0, this.rootDir);
-            insert(rootLocalResource);
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("created the root-metadata-file: " + rootMetadataFileName);
-            }
-            
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Completed the initialization of the local resource repository");
-            }
-            return;
-        }
+                rootMetadataFile.delete();
+                String mountedRootDir;
+                if (rootDir.startsWith(System.getProperty("file.separator"))) {
+                    mountedRootDir = new String(mountPoints[i]
+                            + rootDir.substring(System.getProperty("file.separator").length()));
+                } else {
+                    mountedRootDir = new String(mountPoints[i] + rootDir);
+                }
+                LocalResource rootLocalResource = new LocalResource(ROOT_LOCAL_RESOURCE_ID, rootMetadataFileName, 0, 0,
+                        mountedRootDir);
+                insert(rootLocalResource, i);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("created the root-metadata-file: " + rootMetadataFileName);
+                }
 
-        //#. if the rootMetadataFile exists, read it and set this.rootDir.
-        rootLocalResource = readLocalResource(rootMetadataFile);
-        this.rootDir = (String) rootLocalResource.getResourceObject();
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("The root directory of the local resource repository is " + this.rootDir);
-        }
-
-        //#. load all local resources. 
-        File rootDirFile = new File(this.rootDir);
-        if (!rootDirFile.exists()) {
-            //rootDir may not exist if this node is not the metadata node and doesn't have any user data.
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("The root directory of the local resource repository doesn't exist: there is no local resource.");
-                LOGGER.info("Completed the initialization of the local resource repository");
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Completed the initialization of the local resource repository");
+                }
             }
             return;
         }
@@ -125,27 +120,60 @@
             }
         };
 
-        long maxResourceId = 0;
-        File[] dataverseFileList = rootDirFile.listFiles();
-        if (dataverseFileList == null) {
-            throw new HyracksDataException("Metadata dataverse doesn't exist.");
-        }
-        for (File dataverseFile : dataverseFileList) {
-            if (dataverseFile.isDirectory()) {
-                File[] indexFileList = dataverseFile.listFiles();
-                if (indexFileList != null) {
-                    for (File indexFile : indexFileList) {
-                        if (indexFile.isDirectory()) {
-                            File[] metadataFiles = indexFile.listFiles(filter);
-                            if (metadataFiles != null) {
-                                for (File metadataFile : metadataFiles) {
-                                    LocalResource localResource = readLocalResource(metadataFile);
-                                    id2ResourceMap.put(localResource.getResourceId(), localResource);
-                                    name2ResourceMap.put(localResource.getResourceName(), localResource);
-                                    maxResourceId = Math.max(localResource.getResourceId(), maxResourceId);
-                                    if (LOGGER.isLoggable(Level.INFO)) {
-                                        LOGGER.info("loaded local resource - [id: " + localResource.getResourceId()
-                                                + ", name: " + localResource.getResourceName() + "]");
+        for (int i = 0; i < numIODevices; i++) {
+            String rootMetadataFileName = prepareRootMetaDataFileName(mountPoints[i], nodeId, i) + File.separator
+                    + ROOT_METADATA_FILE_NAME_PREFIX;
+            File rootMetadataFile = new File(rootMetadataFileName);
+            //#. if the rootMetadataFile exists, read it and set this.rootDir.
+            LocalResource rootLocalResource = readLocalResource(rootMetadataFile);
+            String mountedRootDir = (String) rootLocalResource.getResourceObject();
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("The root directory of the local resource repository is " + mountedRootDir);
+            }
+
+            //#. load all local resources. 
+            File rootDirFile = new File(mountedRootDir);
+            if (!rootDirFile.exists()) {
+                //rootDir may not exist if this node is not the metadata node and doesn't have any user data.
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("The root directory of the local resource repository doesn't exist: there is no local resource.");
+                    LOGGER.info("Completed the initialization of the local resource repository");
+                }
+                continue;
+            }
+
+            long maxResourceId = 0;
+            File[] dataverseFileList = rootDirFile.listFiles();
+            if (dataverseFileList == null) {
+                throw new HyracksDataException("Metadata dataverse doesn't exist.");
+            }
+            for (File dataverseFile : dataverseFileList) {
+                if (dataverseFile.isDirectory()) {
+                    File[] indexFileList = dataverseFile.listFiles();
+                    if (indexFileList != null) {
+                        for (File indexFile : indexFileList) {
+                            if (indexFile.isDirectory()) {
+                                File[] ioDevicesList = indexFile.listFiles();
+                                if (ioDevicesList != null) {
+                                    for (File ioDeviceFile : ioDevicesList) {
+                                        if (ioDeviceFile.isDirectory()) {
+                                            File[] metadataFiles = ioDeviceFile.listFiles(filter);
+                                            if (metadataFiles != null) {
+                                                for (File metadataFile : metadataFiles) {
+                                                    LocalResource localResource = readLocalResource(metadataFile);
+                                                    id2ResourceMap.put(localResource.getResourceId(), localResource);
+                                                    name2ResourceMap
+                                                            .put(localResource.getResourceName(), localResource);
+                                                    maxResourceId = Math.max(localResource.getResourceId(),
+                                                            maxResourceId);
+                                                    if (LOGGER.isLoggable(Level.INFO)) {
+                                                        LOGGER.info("loaded local resource - [id: "
+                                                                + localResource.getResourceId() + ", name: "
+                                                                + localResource.getResourceName() + "]");
+                                                    }
+                                                }
+                                            }
+                                        }
                                     }
                                 }
                             }
@@ -153,11 +181,11 @@
                     }
                 }
             }
-        }
-        resourceIdFactory.initId(maxResourceId + 1);
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("The resource id factory is intialized with the value: " + (maxResourceId + 1));
-            LOGGER.info("Completed the initialization of the local resource repository");
+            resourceIdFactory.initId(maxResourceId + 1);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("The resource id factory is intialized with the value: " + (maxResourceId + 1));
+                LOGGER.info("Completed the initialization of the local resource repository");
+            }
         }
     }
 
@@ -172,7 +200,7 @@
     }
 
     @Override
-    public synchronized void insert(LocalResource resource) throws HyracksDataException {
+    public synchronized void insert(LocalResource resource, int ioDeviceId) throws HyracksDataException {
         long id = resource.getResourceId();
 
         if (id2ResourceMap.containsKey(id)) {
@@ -186,8 +214,10 @@
 
         FileOutputStream fos = null;
         ObjectOutputStream oosToFos = null;
+
         try {
-            fos = new FileOutputStream(getFileName(mountPoint, resource.getResourceName(), resource.getResourceId()));
+            fos = new FileOutputStream(getFileName(mountPoints[ioDeviceId], resource.getResourceName(),
+                    resource.getResourceId()));
             oosToFos = new ObjectOutputStream(fos);
             oosToFos.writeObject(resource);
             oosToFos.flush();
@@ -212,26 +242,26 @@
     }
 
     @Override
-    public synchronized void deleteResourceById(long id) throws HyracksDataException {
+    public synchronized void deleteResourceById(long id, int ioDeviceId) throws HyracksDataException {
         LocalResource resource = id2ResourceMap.get(id);
         if (resource == null) {
             throw new HyracksDataException("Resource doesn't exist");
         }
         id2ResourceMap.remove(id);
         name2ResourceMap.remove(resource.getResourceName());
-        File file = new File(getFileName(mountPoint, resource.getResourceName(), resource.getResourceId()));
+        File file = new File(getFileName(mountPoints[ioDeviceId], resource.getResourceName(), resource.getResourceId()));
         file.delete();
     }
 
     @Override
-    public synchronized void deleteResourceByName(String name) throws HyracksDataException {
+    public synchronized void deleteResourceByName(String name, int ioDeviceId) throws HyracksDataException {
         LocalResource resource = name2ResourceMap.get(name);
         if (resource == null) {
             throw new HyracksDataException("Resource doesn't exist");
         }
         id2ResourceMap.remove(resource.getResourceId());
         name2ResourceMap.remove(name);
-        File file = new File(getFileName(mountPoint, resource.getResourceName(), resource.getResourceId()));
+        File file = new File(getFileName(mountPoints[ioDeviceId], resource.getResourceName(), resource.getResourceId()));
         file.delete();
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
index c7efca5..f6847f9 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
@@ -14,11 +14,8 @@
  */
 package edu.uci.ics.asterix.transaction.management.resource;
 
-import java.util.List;
-
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
 
@@ -31,7 +28,6 @@
 
     @Override
     public ILocalResourceRepository createRepository() throws HyracksDataException {
-        List<IODeviceHandle> devices = ioManager.getIODevices();
-        return new PersistentLocalResourceRepository(devices.get(0).getPath().getPath());
+        return new PersistentLocalResourceRepository(ioManager.getIODevices());
     }
-}
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DatasetLockInfo.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DatasetLockInfo.java
index d5e525a..d583352 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DatasetLockInfo.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DatasetLockInfo.java
@@ -437,6 +437,20 @@
 
         return s.toString();
     }
+    
+    public String coreDump() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("\n\t firstUpgrader: " + firstUpgrader);
+        sb.append("\n\t firstWaiter: " + firstWaiter);
+        sb.append("\n\t lastHolder: " + lastHolder);
+        sb.append("\n\t ISCount: " + ISCount);
+        sb.append("\n\t IXCount: " + IXCount);
+        sb.append("\n\t SCount: " + SCount);
+        sb.append("\n\t XCount: " + XCount);
+        sb.append("\n\t entityResourceHT");
+        sb.append(entityResourceHT.prettyPrint());
+        return sb.toString();
+    }
 
     /////////////////////////////////////////////////////////
     //  set/get method for private variable
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java
index 900725b..600beab 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java
@@ -2,8 +2,8 @@
 
 import java.util.HashMap;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.JobId;
 
 /**
  * @author pouria, kisskys Performing a BFS search, upon adding each waiter to a waiting
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityInfoManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityInfoManager.java
index 5d81e8a..1e944bc 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityInfoManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityInfoManager.java
@@ -15,6 +15,8 @@
 
 package edu.uci.ics.asterix.transaction.management.service.locking;
 
+import java.io.IOException;
+import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 
@@ -28,13 +30,12 @@
  */
 public class EntityInfoManager {
 
-    public static final int SHRINK_TIMER_THRESHOLD = 120000; //2min
-
     private ArrayList<ChildEntityInfoArrayManager> pArray;
     private int allocChild; //used to allocate the next free EntityInfo slot.
     private long shrinkTimer;
     private boolean isShrinkTimerOn;
     private int occupiedSlots;
+    private int shrinkTimerThreshold;
 
     //    ////////////////////////////////////////////////
     //    // begin of unit test
@@ -127,12 +128,13 @@
     //    // end of unit test
     //    ////////////////////////////////////////////////
 
-    public EntityInfoManager() {
+    public EntityInfoManager(int shrinkTimerThreshold) {
         pArray = new ArrayList<ChildEntityInfoArrayManager>();
         pArray.add(new ChildEntityInfoArrayManager());
         allocChild = 0;
         occupiedSlots = 0;
         isShrinkTimerOn = false;
+        this.shrinkTimerThreshold = shrinkTimerThreshold;
     }
 
     public int allocate(int jobId, int datasetId, int entityHashVal, byte lockMode) {
@@ -206,7 +208,7 @@
 
         if (size > 1 && size * ChildEntityInfoArrayManager.NUM_OF_SLOTS / usedSlots >= 3) {
             if (isShrinkTimerOn) {
-                if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
+                if (System.currentTimeMillis() - shrinkTimer >= shrinkTimerThreshold) {
                     isShrinkTimerOn = false;
                     return true;
                 }
@@ -272,29 +274,35 @@
             if (child.isDeinitialized()) {
                 continue;
             }
-            s.append("child[" + i + "]: occupiedSlots:" + child.getNumOfOccupiedSlots());
-            s.append(" freeSlotNum:" + child.getFreeSlotNum() + "\n");
-            s.append("\tjid\t").append("did\t").append("PK\t").append("DLM\t").append("DLC\t").append("ELM\t")
-                    .append("ELC\t").append("NEA\t").append("PJR\t").append("NJR\n");
-            for (int j = 0; j < ChildEntityInfoArrayManager.NUM_OF_SLOTS; j++) {
-                s.append(j).append(": ");
-                s.append("\t" + child.getJobId(j));
-                s.append("\t" + child.getDatasetId(j));
-                s.append("\t" + child.getPKHashVal(j));
-                s.append("\t" + child.getDatasetLockMode(j));
-                s.append("\t" + child.getDatasetLockCount(j));
-                s.append("\t" + child.getEntityLockMode(j));
-                s.append("\t" + child.getEntityLockCount(j));
-                s.append("\t" + child.getNextEntityActor(j));
-                s.append("\t" + child.getPrevJobResource(j));
-                s.append("\t" + child.getNextJobResource(j));
-                //s.append("\t" + child.getNextDatasetActor(j));
-                s.append("\n");
-            }
-            s.append("\n");
+            s.append("child[" + i + "]");
+            s.append(child.prettyPrint());
         }
         return s.toString();
     }
+    
+    public void coreDump(OutputStream os) {
+        StringBuilder sb = new StringBuilder("\n\t########### EntityLockInfoManager Status #############\n");
+        int size = pArray.size();
+        ChildEntityInfoArrayManager child;
+
+        sb.append("Number of Child: " + size + "\n"); 
+        for (int i = 0; i < size; i++) {
+            try {
+                child = pArray.get(i);
+                sb.append("child[" + i + "]");
+                sb.append(child.prettyPrint());
+                
+                os.write(sb.toString().getBytes());
+            } catch (IOException e) {
+                //ignore IOException
+            }
+            sb = new StringBuilder();
+        }
+    }
+    
+    public int getShrinkTimerThreshold() {
+        return shrinkTimerThreshold;
+    }
 
     public void initEntityInfo(int slotNum, int jobId, int datasetId, int PKHashVal, byte lockMode) {
         pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).initEntityInfo(
@@ -567,6 +575,29 @@
     public int getFreeSlotNum() {
         return freeSlotNum;
     }
+    
+    public String prettyPrint() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("\n\toccupiedSlots:" + getNumOfOccupiedSlots());
+        sb.append("\n\tfreeSlotNum:" + getFreeSlotNum() + "\n");
+        sb.append("\tjid\t").append("did\t").append("PK\t").append("DLM\t").append("DLC\t").append("ELM\t")
+                .append("ELC\t").append("NEA\t").append("PJR\t").append("NJR\n");
+        for (int j = 0; j < ChildEntityInfoArrayManager.NUM_OF_SLOTS; j++) {
+            sb.append(j).append(": ");
+            sb.append("\t" + getJobId(j));
+            sb.append("\t" + getDatasetId(j));
+            sb.append("\t" + getPKHashVal(j));
+            sb.append("\t" + getDatasetLockMode(j));
+            sb.append("\t" + getDatasetLockCount(j));
+            sb.append("\t" + getEntityLockMode(j));
+            sb.append("\t" + getEntityLockCount(j));
+            sb.append("\t" + getNextEntityActor(j));
+            sb.append("\t" + getPrevJobResource(j));
+            sb.append("\t" + getNextJobResource(j));
+            sb.append("\n");
+        }
+        return sb.toString();
+    }
 
     //////////////////////////////////////////////////////////////////
     //   set/get method for each field of EntityInfo plus freeSlot
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java
index ca00aa2..2fae460 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java
@@ -15,6 +15,8 @@
 
 package edu.uci.ics.asterix.transaction.management.service.locking;
 
+import java.io.IOException;
+import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 
@@ -271,22 +273,35 @@
             if (child.isDeinitialized()) {
                 continue;
             }
-            s.append("child[" + i + "]: occupiedSlots:" + child.getNumOfOccupiedSlots());
-            s.append(" freeSlotNum:" + child.getFreeSlotNum() + "\n");
-            s.append("\tX\t").append("S\t").append("LH\t").append("FW\t").append("UP\n");
-            for (int j = 0; j < ChildEntityLockInfoArrayManager.NUM_OF_SLOTS; j++) {
-                s.append(j).append(": ");
-                s.append("\t" + child.getXCount(j));
-                s.append("\t" + child.getSCount(j));
-                s.append("\t" + child.getLastHolder(j));
-                s.append("\t" + child.getFirstWaiter(j));
-                s.append("\t" + child.getUpgrader(j));
-                s.append("\n");
-            }
-            s.append("\n");
+            s.append("child[" + i + "]");
+            s.append(child.prettyPrint());
         }
         return s.toString();
     }
+    
+    public void coreDump(OutputStream os) {
+        StringBuilder sb = new StringBuilder("\n\t########### EntityLockInfoManager Status #############\n");
+        int size = pArray.size();
+        ChildEntityLockInfoArrayManager child;
+
+        sb.append("Number of Child: " + size + "\n"); 
+        for (int i = 0; i < size; i++) {
+            try {
+                child = pArray.get(i);
+                sb.append("child[" + i + "]");
+                sb.append(child.prettyPrint());
+                
+                os.write(sb.toString().getBytes());
+            } catch (IOException e) {
+                //ignore IOException
+            }
+            sb = new StringBuilder();
+        }
+    }
+    
+    public int getShrinkTimerThreshold() {
+        return SHRINK_TIMER_THRESHOLD;
+    }
 
     //debugging method
     public String printWaiters(int slotNum) {
@@ -736,6 +751,23 @@
     public int getFreeSlotNum() {
         return freeSlotNum;
     }
+    
+    public String prettyPrint() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("\n\toccupiedSlots:" + getNumOfOccupiedSlots());
+        sb.append("\n\tfreeSlotNum:" + getFreeSlotNum());
+        sb.append("\n\tX\t").append("S\t").append("LH\t").append("FW\t").append("UP\n");
+        for (int j = 0; j < ChildEntityLockInfoArrayManager.NUM_OF_SLOTS; j++) {
+            sb.append(j).append(": ");
+            sb.append("\t" + getXCount(j));
+            sb.append("\t" + getSCount(j));
+            sb.append("\t" + getLastHolder(j));
+            sb.append("\t" + getFirstWaiter(j));
+            sb.append("\t" + getUpgrader(j));
+            sb.append("\n");
+        }
+        return sb.toString();
+    }
 
     //////////////////////////////////////////////////////////////////
     //   set/get method for each field of EntityLockInfo plus freeSlot
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java
index 4f01f37..86eb02b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java
@@ -1,19 +1,19 @@
 package edu.uci.ics.asterix.transaction.management.service.locking;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 
 public class JobInfo {
     private EntityInfoManager entityInfoManager;
     private LockWaiterManager lockWaiterManager;
-    private TransactionContext jobCtx;
+    private ITransactionContext jobCtx;
     private int lastHoldingResource; //resource(entity or dataset) which is held by this job lastly
     private int firstWaitingResource; //resource(entity or dataset) which this job is waiting for
     private int upgradingResource; //resource(entity or dataset) which this job is waiting for to upgrade
 
     private PrimitiveIntHashMap datasetISLockHT; //used for keeping dataset-granule-lock's count acquired by this job. 
 
-    public JobInfo(EntityInfoManager entityInfoManager, LockWaiterManager lockWaiterManager, TransactionContext txnCtx) {
+    public JobInfo(EntityInfoManager entityInfoManager, LockWaiterManager lockWaiterManager, ITransactionContext txnCtx) {
         this.entityInfoManager = entityInfoManager;
         this.lockWaiterManager = lockWaiterManager;
         this.jobCtx = txnCtx;
@@ -192,9 +192,9 @@
         }
     }
 
-    public void decreaseDatasetISLockCount(int datasetId) {
+    public void decreaseDatasetISLockCount(int datasetId, int entityToDatasetLockEscalationThreshold) {
         int count = datasetISLockHT.get(datasetId);
-        if (count >= LockManager.ESCALATE_TRHESHOLD_ENTITY_TO_DATASET) {
+        if (count >= entityToDatasetLockEscalationThreshold) {
             //do not decrease the count since it is already escalated.
         } else if (count > 1) {
             datasetISLockHT.upsert(datasetId, count - 1);
@@ -275,6 +275,17 @@
         }
         return s.toString();
     }
+    
+    public String coreDump() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("\n\t datasetISLockHT");
+        sb.append(datasetISLockHT.prettyPrint());
+        sb.append("\n\t firstWaitingResource: " + firstWaitingResource);
+        sb.append("\n\t lastHoldingResource: " + lastHoldingResource);
+        sb.append("\n\t upgradingResource: " + upgradingResource);
+        sb.append("\n\t jobCtx.jobId: " + jobCtx.getJobId());
+        return sb.toString();
+    }
 
     /////////////////////////////////////////////////////////
     //  set/get method for private variable
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
index a354d2a..9768198 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
@@ -15,23 +15,29 @@
 
 package edu.uci.ics.asterix.transaction.management.service.locking;
 
+import java.io.IOException;
+import java.io.OutputStream;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager.TransactionState;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
 
 /**
  * An implementation of the ILockManager interface for the
@@ -41,7 +47,7 @@
  * @author pouria, kisskys
  */
 
-public class LockManager implements ILockManager {
+public class LockManager implements ILockManager, ILifeCycleComponent {
 
     public static final boolean IS_DEBUG_MODE = false;//true
     //This variable indicates that the dataset granule X lock request is allowed when 
@@ -51,8 +57,6 @@
     public static final boolean ALLOW_DATASET_GRANULE_X_LOCK_WITH_OTHER_CONCURRENT_LOCK_REQUESTS = false;
 
     public static final boolean ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET = true;
-    //Threshold must be greater than 1 and should be reasonably large enough not to escalate too soon.
-    public static final int ESCALATE_TRHESHOLD_ENTITY_TO_DATASET = 1000;
     private static final int DO_ESCALATE = 0;
     private static final int ESCALATED = 1;
     private static final int DONOT_ESCALATE = 2;
@@ -90,7 +94,7 @@
         this.waiterLatch = new ReentrantReadWriteLock(true);
         this.jobHT = new HashMap<JobId, JobInfo>();
         this.datasetResourceHT = new HashMap<DatasetId, DatasetLockInfo>();
-        this.entityInfoManager = new EntityInfoManager();
+        this.entityInfoManager = new EntityInfoManager(txnSubsystem.getTransactionProperties().getLockManagerShrinkTimer());
         this.lockWaiterManager = new LockWaiterManager();
         this.entityLockInfoManager = new EntityLockInfoManager(entityInfoManager, lockWaiterManager);
         this.deadlockDetector = new DeadlockDetector(jobHT, datasetResourceHT, entityLockInfoManager,
@@ -105,14 +109,18 @@
             this.lockRequestTracker = new LockRequestTracker();
         }
     }
+     
+    public AsterixTransactionProperties getTransactionProperties() {
+        return this.txnSubsystem.getTransactionProperties();
+    }
 
     @Override
-    public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+    public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
             throws ACIDException {
         internalLock(datasetId, entityHashValue, lockMode, txnContext, false);
     }
 
-    private void internalLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext,
+    private void internalLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext,
             boolean isInstant) throws ACIDException {
 
         JobId jobId = txnContext.getJobId();
@@ -192,7 +200,7 @@
                         if (doEscalate) {
                             throw new IllegalStateException(
                                     "ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
-                                            + ESCALATE_TRHESHOLD_ENTITY_TO_DATASET);
+                                            + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
                         }
                     }
                 }
@@ -252,7 +260,7 @@
         return;
     }
 
-    private void releaseDatasetISLocks(JobInfo jobInfo, JobId jobId, DatasetId datasetId, TransactionContext txnContext)
+    private void releaseDatasetISLocks(JobInfo jobInfo, JobId jobId, DatasetId datasetId, ITransactionContext txnContext)
             throws ACIDException {
         int entityInfo;
         int prevEntityInfo;
@@ -286,25 +294,25 @@
         }
 
         int count = jobInfo.getDatasetISLockCount(datasetId);
-        if (count == ESCALATE_TRHESHOLD_ENTITY_TO_DATASET) {
+        if (count == txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold()) {
             return DO_ESCALATE;
-        } else if (count > ESCALATE_TRHESHOLD_ENTITY_TO_DATASET) {
+        } else if (count > txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold()) {
             return ESCALATED;
         } else {
             return DONOT_ESCALATE;
         }
     }
 
-    private void validateJob(TransactionContext txnContext) throws ACIDException {
+    private void validateJob(ITransactionContext txnContext) throws ACIDException {
         if (txnContext.getTxnState() == TransactionState.ABORTED) {
             throw new ACIDException("" + txnContext.getJobId() + " is in ABORTED state.");
-        } else if (txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
+        } else if (txnContext.getStatus() == ITransactionContext.TIMED_OUT_STATUS) {
             requestAbort(txnContext);
         }
     }
 
     private int lockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            TransactionContext txnContext) throws ACIDException {
+            ITransactionContext txnContext) throws ACIDException {
         JobId jobId = txnContext.getJobId();
         int jId = jobId.getId(); //int-type jobId
         int dId = datasetId.getId(); //int-type datasetId
@@ -525,7 +533,7 @@
     }
 
     private void lockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            int entityInfoFromDLockInfo, TransactionContext txnContext) throws ACIDException {
+            int entityInfoFromDLockInfo, ITransactionContext txnContext) throws ACIDException {
         JobId jobId = txnContext.getJobId();
         int jId = jobId.getId(); //int-type jobId
         int waiterObjId;
@@ -632,22 +640,23 @@
     }
 
     @Override
-    public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext) throws ACIDException {
-        internalUnlock(datasetId, entityHashValue, txnContext, false, false);
+    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
+            throws ACIDException {
+        return internalUnlock(datasetId, entityHashValue, txnContext, false, false);
     }
 
     @Override
-    public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext, boolean commitFlag)
+    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
             throws ACIDException {
-        internalUnlock(datasetId, entityHashValue, txnContext, false, commitFlag);
+        return internalUnlock(datasetId, entityHashValue, txnContext, false, commitFlag);
     }
 
-    private void instantUnlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext)
+    private void instantUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
             throws ACIDException {
         internalUnlock(datasetId, entityHashValue, txnContext, true, false);
     }
 
-    private void internalUnlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext,
+    private boolean internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
             boolean isInstant, boolean commitFlag) throws ACIDException {
         JobId jobId = txnContext.getJobId();
         int eLockInfo = -1;
@@ -656,6 +665,7 @@
         int entityInfo = -1;
         byte datasetLockMode;
 
+        boolean lockCountIsZero = false;
         if (IS_DEBUG_MODE) {
             if (entityHashValue == -1) {
                 throw new UnsupportedOperationException(
@@ -703,6 +713,7 @@
 
             if (entityInfoManager.getEntityLockCount(entityInfo) == 0
                     && entityInfoManager.getDatasetLockCount(entityInfo) == 0) {
+                lockCountIsZero = true;
                 int threadCount = 0; //number of threads(in the same job) waiting on the same resource 
                 int waiterObjId = jobInfo.getFirstWaitingResource();
                 int waitingEntityInfo;
@@ -712,7 +723,7 @@
                 //This code should be taken care properly when there is a way to avoid doubling memory space for txnIds.
                 //This commit log is written here in order to avoid increasing the memory space for managing transactionIds
                 if (commitFlag) {
-                    if (txnContext.getTransactionType().equals(TransactionContext.TransactionType.READ_WRITE)) {
+                    if (txnContext.getTransactionType().equals(ITransactionContext.TransactionType.READ_WRITE)) {
                         try {
                             txnSubsystem.getLogManager().log(LogType.ENTITY_COMMIT, txnContext, datasetId.getId(),
                                     entityHashValue, -1, (byte) 0, 0, null, null, logicalLogLocator);
@@ -771,7 +782,7 @@
 
             if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
                 if (!isInstant && datasetLockMode == LockMode.IS) {
-                    jobInfo.decreaseDatasetISLockCount(datasetId.getId());
+                    jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
                 }
             }
 
@@ -782,10 +793,11 @@
         } finally {
             unlatchLockTable();
         }
+        return lockCountIsZero;
     }
 
     @Override
-    public void releaseLocks(TransactionContext txnContext) throws ACIDException {
+    public void releaseLocks(ITransactionContext txnContext) throws ACIDException {
         LockWaiter waiterObj;
         int entityInfo;
         int prevEntityInfo;
@@ -962,7 +974,7 @@
             jobHT.remove(jobId);
 
             if (existWaiter) {
-                txnContext.setStatus(TransactionContext.TIMED_OUT_STATUS);
+                txnContext.setStatus(ITransactionContext.TIMED_OUT_STATUS);
                 txnContext.setTxnState(TransactionState.ABORTED);
             }
 
@@ -976,7 +988,7 @@
     }
 
     @Override
-    public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+    public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
             throws ACIDException {
 
         //        try {
@@ -990,19 +1002,19 @@
     }
 
     @Override
-    public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+    public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
             throws ACIDException {
         return internalTryLock(datasetId, entityHashValue, lockMode, txnContext, false);
     }
 
     @Override
-    public boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
-            throws ACIDException {
+    public boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
+            ITransactionContext txnContext) throws ACIDException {
         return internalInstantTryLock(datasetId, entityHashValue, lockMode, txnContext);
     }
 
     private boolean internalInstantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
-            TransactionContext txnContext) throws ACIDException {
+            ITransactionContext txnContext) throws ACIDException {
         DatasetLockInfo dLockInfo = null;
         boolean isSuccess = true;
 
@@ -1065,7 +1077,7 @@
     }
 
     private boolean instantTryLockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            TransactionContext txnContext, DatasetLockInfo dLockInfo, byte datasetLockMode) throws ACIDException {
+            ITransactionContext txnContext, DatasetLockInfo dLockInfo, byte datasetLockMode) throws ACIDException {
         JobId jobId = txnContext.getJobId();
         int jId = jobId.getId(); //int-type jobId
         int dId = datasetId.getId(); //int-type datasetId
@@ -1144,7 +1156,7 @@
     }
 
     private boolean instantTryLockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            TransactionContext txnContext, DatasetLockInfo dLockInfo) throws ACIDException {
+            ITransactionContext txnContext, DatasetLockInfo dLockInfo) throws ACIDException {
         JobId jobId = txnContext.getJobId();
         int jId = jobId.getId(); //int-type jobId
         int waiterObjId;
@@ -1205,7 +1217,7 @@
     }
 
     private boolean internalTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
-            TransactionContext txnContext, boolean isInstant) throws ACIDException {
+            ITransactionContext txnContext, boolean isInstant) throws ACIDException {
         JobId jobId = txnContext.getJobId();
         int jId = jobId.getId(); //int-type jobId
         int dId = datasetId.getId(); //int-type datasetId
@@ -1285,7 +1297,7 @@
                             //We don't want to allow the lock escalation when there is a first lock request on a dataset. 
                             throw new IllegalStateException(
                                     "ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
-                                            + ESCALATE_TRHESHOLD_ENTITY_TO_DATASET);
+                                            + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
                         }
                     }
                 }
@@ -1352,7 +1364,7 @@
     }
 
     private void trackLockRequest(String msg, int requestType, DatasetId datasetIdObj, int entityHashValue,
-            byte lockMode, TransactionContext txnContext, DatasetLockInfo dLockInfo, int eLockInfo) {
+            byte lockMode, ITransactionContext txnContext, DatasetLockInfo dLockInfo, int eLockInfo) {
         StringBuilder s = new StringBuilder();
         LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, datasetIdObj,
                 entityHashValue, lockMode, txnContext);
@@ -1425,7 +1437,7 @@
     }
 
     private void revertTryLockDatasetGranuleOperation(DatasetId datasetId, int entityHashValue, byte lockMode,
-            int entityInfo, TransactionContext txnContext) {
+            int entityInfo, ITransactionContext txnContext) {
         JobId jobId = txnContext.getJobId();
         DatasetLockInfo dLockInfo;
         JobInfo jobInfo;
@@ -1483,7 +1495,7 @@
     }
 
     private int tryLockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            TransactionContext txnContext) throws ACIDException {
+            ITransactionContext txnContext) throws ACIDException {
         JobId jobId = txnContext.getJobId();
         int jId = jobId.getId(); //int-type jobId
         int dId = datasetId.getId(); //int-type datasetId
@@ -1645,7 +1657,7 @@
     }
 
     private boolean tryLockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            int entityInfoFromDLockInfo, TransactionContext txnContext) throws ACIDException {
+            int entityInfoFromDLockInfo, ITransactionContext txnContext) throws ACIDException {
         JobId jobId = txnContext.getJobId();
         int jId = jobId.getId(); //int-type jobId
         int waiterObjId;
@@ -1735,7 +1747,7 @@
     }
 
     private int handleLockWaiter(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isUpgrade,
-            boolean isDatasetLockInfo, TransactionContext txnContext, JobInfo jobInfo, int duplicatedWaiterObjId)
+            boolean isDatasetLockInfo, ITransactionContext txnContext, JobInfo jobInfo, int duplicatedWaiterObjId)
             throws ACIDException {
         int waiterId = -1;
         LockWaiter waiter;
@@ -1812,7 +1824,7 @@
             //waiter woke up -> remove/deallocate waiter object and abort if timeout
             latchLockTable();
 
-            if (txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS || waiter.isVictim()) {
+            if (txnContext.getStatus() == ITransactionContext.TIMED_OUT_STATUS || waiter.isVictim()) {
                 requestAbort(txnContext);
             }
 
@@ -1868,9 +1880,9 @@
         return deadlockDetector.isSafeToAdd(dLockInfo, eLockInfo, entityInfo, isDatasetLockInfo, isUpgrade);
     }
 
-    private void requestAbort(TransactionContext txnContext) throws ACIDException {
-        txnContext.setStatus(TransactionContext.TIMED_OUT_STATUS);
-        txnContext.setStartWaitTime(TransactionContext.INVALID_TIME);
+    private void requestAbort(ITransactionContext txnContext) throws ACIDException {
+        txnContext.setStatus(ITransactionContext.TIMED_OUT_STATUS);
+        txnContext.setStartWaitTime(ITransactionContext.INVALID_TIME);
         throw new ACIDException("Transaction " + txnContext.getJobId()
                 + " should abort (requested by the Lock Manager)");
     }
@@ -2045,6 +2057,182 @@
             unlatchLockTable();
         }
     }
+
+    @Override
+    public void start() {
+        //no op
+    }
+
+    @Override
+    public void stop(boolean dumpState, OutputStream os) {
+        if (dumpState) {
+
+            //#. dump Configurable Variables
+            dumpConfVars(os);
+
+            //#. dump jobHT
+            dumpJobInfo(os);
+
+            //#. dump datasetResourceHT
+            dumpDatasetLockInfo(os);
+
+            //#. dump entityLockInfoManager
+            dumpEntityLockInfo(os);
+
+            //#. dump entityInfoManager
+            dumpEntityInfo(os);
+
+            //#. dump lockWaiterManager
+
+            dumpLockWaiterInfo(os);
+            try {
+                os.flush();
+            } catch (IOException e) {
+                //ignore
+            }
+        }
+    }
+
+    private void dumpConfVars(OutputStream os) {
+        try {
+            StringBuilder sb = new StringBuilder();
+            sb.append("\n>>dump_begin\t>>----- [ConfVars] -----");
+            sb.append("\nESCALATE_TRHESHOLD_ENTITY_TO_DATASET: " + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+            sb.append("\nSHRINK_TIMER_THRESHOLD (entityLockInfoManager): "
+                    + entityLockInfoManager.getShrinkTimerThreshold());
+            sb.append("\nSHRINK_TIMER_THRESHOLD (entityInfoManager): " + entityInfoManager.getShrinkTimerThreshold());
+            sb.append("\nSHRINK_TIMER_THRESHOLD (lockWaiterManager): " + lockWaiterManager.getShrinkTimerThreshold());
+            sb.append("\n>>dump_end\t>>----- [ConfVars] -----\n");
+            os.write(sb.toString().getBytes());
+        } catch (Exception e) {
+            //ignore exception and continue dumping as much as possible.
+            if (IS_DEBUG_MODE) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    private void dumpJobInfo(OutputStream os) {
+        JobId jobId;
+        JobInfo jobInfo;
+        StringBuilder sb = new StringBuilder();
+
+        try {
+            sb.append("\n>>dump_begin\t>>----- [JobInfo] -----");
+            Set<Map.Entry<JobId, JobInfo>> entrySet = jobHT.entrySet();
+            if (entrySet != null) {
+                for (Map.Entry<JobId, JobInfo> entry : entrySet) {
+                    if (entry != null) {
+                        jobId = entry.getKey();
+                        if (jobId != null) {
+                            sb.append("\n" + jobId);
+                        } else {
+                            sb.append("\nJID:null");
+                        }
+
+                        jobInfo = entry.getValue();
+                        if (jobInfo != null) {
+                            sb.append(jobInfo.coreDump());
+                        } else {
+                            sb.append("\nJobInfo:null");
+                        }
+                    }
+                }
+            }
+            sb.append("\n>>dump_end\t>>----- [JobInfo] -----\n");
+            os.write(sb.toString().getBytes());
+        } catch (Exception e) {
+            //ignore exception and continue dumping as much as possible.
+            if (IS_DEBUG_MODE) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    private void dumpDatasetLockInfo(OutputStream os) {
+        DatasetId datasetId;
+        DatasetLockInfo datasetLockInfo;
+        StringBuilder sb = new StringBuilder();
+
+        try {
+            sb.append("\n>>dump_begin\t>>----- [DatasetLockInfo] -----");
+            Set<Map.Entry<DatasetId, DatasetLockInfo>> entrySet = datasetResourceHT.entrySet();
+            if (entrySet != null) {
+                for (Map.Entry<DatasetId, DatasetLockInfo> entry : entrySet) {
+                    if (entry != null) {
+                        datasetId = entry.getKey();
+                        if (datasetId != null) {
+                            sb.append("\nDatasetId:" + datasetId.getId());
+                        } else {
+                            sb.append("\nDatasetId:null");
+                        }
+
+                        datasetLockInfo = entry.getValue();
+                        if (datasetLockInfo != null) {
+                            sb.append(datasetLockInfo.coreDump());
+                        } else {
+                            sb.append("\nDatasetLockInfo:null");
+                        }
+                    }
+                    sb.append("\n>>dump_end\t>>----- [DatasetLockInfo] -----\n");
+                    os.write(sb.toString().getBytes());
+
+                    //create a new sb to avoid possible OOM exception
+                    sb = new StringBuilder();
+                }
+            }
+        } catch (Exception e) {
+            //ignore exception and continue dumping as much as possible.
+            if (IS_DEBUG_MODE) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    private void dumpEntityLockInfo(OutputStream os) {
+        StringBuilder sb = new StringBuilder();
+        try {
+            sb.append("\n>>dump_begin\t>>----- [EntityLockInfo] -----");
+            entityLockInfoManager.coreDump(os);
+            sb.append("\n>>dump_end\t>>----- [EntityLockInfo] -----\n");
+            os.write(sb.toString().getBytes());
+        } catch (Exception e) {
+            //ignore exception and continue dumping as much as possible.
+            if (IS_DEBUG_MODE) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    private void dumpEntityInfo(OutputStream os) {
+        StringBuilder sb = new StringBuilder();
+        try {
+            sb.append("\n>>dump_begin\t>>----- [EntityInfo] -----");
+            entityInfoManager.coreDump(os);
+            sb.append("\n>>dump_end\t>>----- [EntityInfo] -----\n");
+            os.write(sb.toString().getBytes());
+        } catch (Exception e) {
+            //ignore exception and continue dumping as much as possible.
+            if (IS_DEBUG_MODE) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    private void dumpLockWaiterInfo(OutputStream os) {
+        StringBuilder sb = new StringBuilder();
+        try {
+            sb.append("\n>>dump_begin\t>>----- [LockWaiterInfo] -----");
+            lockWaiterManager.coreDump(os);
+            sb.append("\n>>dump_end\t>>----- [LockWaiterInfo] -----\n");
+            os.write(sb.toString().getBytes());
+        } catch (Exception e) {
+            //ignore exception and continue dumping as much as possible.
+            if (IS_DEBUG_MODE) {
+                e.printStackTrace();
+            }
+        }
+    }
 }
 
 class ConsecutiveWakeupContext {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
index c3b47bc..02fda06 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
@@ -7,17 +7,25 @@
 import java.util.NoSuchElementException;
 import java.util.Scanner;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager.TransactionState;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
+import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 public class LockManagerDeterministicUnitTest {
 
-    public static void main(String args[]) throws ACIDException, IOException {
+    public static void main(String args[]) throws ACIDException, IOException, AsterixException {
         //initialize controller thread
         String requestFileName = new String(
                 "src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile");
@@ -38,8 +46,9 @@
     String requestFileName;
     long defaultWaitTime;
 
-    public LockRequestController(String requestFileName) throws ACIDException {
-        this.txnProvider = new TransactionSubsystem("LockManagerPredefinedUnitTest", null);;
+    public LockRequestController(String requestFileName) throws ACIDException, AsterixException {
+        this.txnProvider = new TransactionSubsystem("LockManagerPredefinedUnitTest", null,
+                new AsterixTransactionProperties(new AsterixPropertiesAccessor()));
         this.workerReadyQueue = new WorkerReadyQueue();
         this.requestList = new ArrayList<LockRequest>();
         this.expectedResultList = new ArrayList<ArrayList<Integer>>();
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
index 89a15ef..a942325 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
@@ -3,10 +3,15 @@
 import java.util.ArrayList;
 import java.util.Random;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager.TransactionState;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
@@ -26,9 +31,10 @@
     private static int jobId = 0;
     private static Random rand;
 
-    public static void main(String args[]) throws ACIDException {
+    public static void main(String args[]) throws ACIDException, AsterixException {
         int i;
-        TransactionSubsystem txnProvider = new TransactionSubsystem("LockManagerRandomUnitTest", null);
+        TransactionSubsystem txnProvider = new TransactionSubsystem("LockManagerRandomUnitTest", null,
+                new AsterixTransactionProperties(new AsterixPropertiesAccessor()));
         rand = new Random(System.currentTimeMillis());
         for (i = 0; i < MAX_NUM_OF_ENTITY_LOCK_JOB; i++) {
             System.out.println("Creating " + i + "th EntityLockJob..");
@@ -502,14 +508,14 @@
     public DatasetId datasetIdObj;
     public int entityHashValue;
     public byte lockMode;
-    public TransactionContext txnContext;
+    public ITransactionContext txnContext;
     public boolean isUpgrade;
     public boolean isTryLockFailed;
     public long requestTime;
     public String threadName;
 
     public LockRequest(String threadName, int requestType, DatasetId datasetIdObj, int entityHashValue, byte lockMode,
-            TransactionContext txnContext) {
+            ITransactionContext txnContext) {
         this.requestType = requestType;
         this.datasetIdObj = datasetIdObj;
         this.entityHashValue = entityHashValue;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockMgrLatchHandlerException.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockMgrLatchHandlerException.java
index 05a582c..84ba407 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockMgrLatchHandlerException.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockMgrLatchHandlerException.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.asterix.transaction.management.service.locking;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 
 public class LockMgrLatchHandlerException extends ACIDException {
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java
index bd414de..c6fcbd5 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java
@@ -15,6 +15,8 @@
 
 package edu.uci.ics.asterix.transaction.management.service.locking;
 
+import java.io.IOException;
+import java.io.OutputStream;
 import java.util.ArrayList;
 
 /**
@@ -258,28 +260,42 @@
         StringBuilder s = new StringBuilder("\n########### LockWaiterManager Status #############\n");
         int size = pArray.size();
         ChildLockWaiterArrayManager child;
-        LockWaiter waiter;
 
         for (int i = 0; i < size; i++) {
             child = pArray.get(i);
             if (child.isDeinitialized()) {
                 continue;
             }
-            s.append("child[" + i + "]: occupiedSlots:" + child.getNumOfOccupiedSlots());
-            s.append(" freeSlotNum:" + child.getFreeSlotNum() + "\n");
-            for (int j = 0; j < ChildLockWaiterArrayManager.NUM_OF_SLOTS; j++) {
-                waiter = child.getLockWaiter(j);
-                s.append(j).append(": ");
-                s.append("\t" + waiter.getEntityInfoSlot());
-                s.append("\t" + waiter.needWait());
-                s.append("\t" + waiter.isVictim());
-                s.append("\n");
-            }
-            s.append("\n");
+            s.append("child[" + i + "]");
+            s.append(child.prettyPrint());
         }
         return s.toString();
     }
     
+    public void coreDump(OutputStream os) {
+        StringBuilder sb = new StringBuilder("\n########### LockWaiterManager Status #############\n");
+        int size = pArray.size();
+        ChildLockWaiterArrayManager child;
+
+        sb.append("Number of Child: " + size + "\n"); 
+        for (int i = 0; i < size; i++) {
+            try {
+                child = pArray.get(i);
+                sb.append("child[" + i + "]");
+                sb.append(child.prettyPrint());
+                
+                os.write(sb.toString().getBytes());
+            } catch (IOException e) {
+                //ignore IOException
+            }
+            sb = new StringBuilder();
+        }
+    }
+    
+    public int getShrinkTimerThreshold() {
+        return SHRINK_TIMER_THRESHOLD;
+    }
+    
     public LockWaiter getLockWaiter(int slotNum) {
         return pArray.get(slotNum / ChildLockWaiterArrayManager.NUM_OF_SLOTS).getLockWaiter(
                 slotNum % ChildLockWaiterArrayManager.NUM_OF_SLOTS);
@@ -364,4 +380,20 @@
     public int getFreeSlotNum() {
         return freeSlotNum;
     }
+    
+    public String prettyPrint() {
+        LockWaiter waiter;
+        StringBuilder sb = new StringBuilder();
+        sb.append("\n\toccupiedSlots:" + getNumOfOccupiedSlots());
+        sb.append("\n\tfreeSlotNum:" + getFreeSlotNum() + "\n");
+        for (int j = 0; j < ChildLockWaiterArrayManager.NUM_OF_SLOTS; j++) {
+            waiter = getLockWaiter(j);
+            sb.append(j).append(": ");
+            sb.append("\t" + waiter.getEntityInfoSlot());
+            sb.append("\t" + waiter.needWait());
+            sb.append("\t" + waiter.isVictim());
+            sb.append("\n");
+        }
+        return sb.toString();
+    }
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java
index 6bc8c6b..7cbd193 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java
@@ -2,7 +2,8 @@
 
 import java.util.LinkedList;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 
 /**
  * @author pouria, kisskys
@@ -14,19 +15,21 @@
  */
 
 public class TimeOutDetector {
-    static final long TIME_OUT_THRESHOLD = 60000;
-    static final long SWEEP_PERIOD = 10000;//120000;
 
     LockManager lockMgr;
     Thread trigger;
     LinkedList<LockWaiter> victimList;
+    int timeoutThreshold;
+    int sweepThreshold;
 
     public TimeOutDetector(LockManager lockMgr) {
         this.victimList = new LinkedList<LockWaiter>();
         this.lockMgr = lockMgr;
         this.trigger = new Thread(new TimeoutTrigger(this));
+        this.timeoutThreshold = lockMgr.getTransactionProperties().getTimeoutWaitThreshold();
+        this.sweepThreshold = lockMgr.getTransactionProperties().getTimeoutSweepThreshold();
         trigger.setDaemon(true);
-        trigger.start();
+        AsterixThreadExecutor.INSTANCE.execute(trigger);
     }
 
     public void sweep() throws ACIDException {
@@ -38,7 +41,7 @@
     }
 
     public void checkAndSetVictim(LockWaiter waiterObj) {
-        if (System.currentTimeMillis() - waiterObj.getBeginWaitTime() >= TIME_OUT_THRESHOLD) {
+        if (System.currentTimeMillis() - waiterObj.getBeginWaitTime() >= timeoutThreshold) {
             waiterObj.setVictim(true);
             waiterObj.setWait(false);
             victimList.add(waiterObj);
@@ -67,7 +70,7 @@
     public void run() {
         while (true) {
             try {
-                Thread.sleep(TimeOutDetector.SWEEP_PERIOD);
+                Thread.sleep(owner.sweepThreshold);
                 owner.sweep(); // Trigger the timeout detector (the owner) to
                                // initiate sweep
             } catch (InterruptedException e) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
index 23a60f7..284b081 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
@@ -17,6 +17,8 @@
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
+import edu.uci.ics.asterix.common.transactions.IBuffer;
+
 /**
  * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
  * transaction support. This class is not "thread-safe". For performance
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
index 91ce7ed..0d8448c 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
@@ -16,6 +16,8 @@
 
 import java.util.zip.CRC32;
 
+import edu.uci.ics.asterix.common.transactions.IBuffer;
+
 /**
  * A utility class for doing bit level operations such as forming checksum or
  * converting between Integer and byte array. Used extensively during writing
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
index 6bb644d..e9ec4fa 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
@@ -17,6 +17,9 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import edu.uci.ics.asterix.common.transactions.IBuffer;
+import edu.uci.ics.asterix.common.transactions.ILogFilter;
+
 /*
  *  A generic filter that encompasses one or more filters (see @ILogFiler) that may be applied when selectively retrieving logs.
  *  The contained filters are assumed to form a conjunction.   
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java
deleted file mode 100644
index e26a3cc..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-
-/**
- * An interface providing call back APIs that are invoked {@link ILogManager} for providing the content for the log record and doing any pre/post
- * processing.
- */
-public interface ILogger {
-
-    public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException;
-
-    public void log(TransactionContext context, final LogicalLogLocator logicalLogLocator, int logContentSize,
-            ReusableLogContentObject reusableLogContentObject) throws ACIDException;
-
-    public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException;
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
index d69a36e..ca30124 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
@@ -18,11 +18,14 @@
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.ICloseable;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ICloseable;
+import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -53,14 +56,14 @@
         this.tupleWriter = new SimpleTupleWriter();
     }
 
-    public synchronized void close(TransactionContext context) {
+    public synchronized void close(ITransactionContext context) {
         ReusableLogContentObjectRepository txnThreadStateRepository = (ReusableLogContentObjectRepository) jobId2ReusableLogContentObjectRepositoryMap
                 .get(context.getJobId());
         txnThreadStateRepository.remove(Thread.currentThread().getId());
         jobId2ReusableLogContentObjectRepositoryMap.remove(context.getJobId());
     }
 
-    public void generateLogRecord(TransactionSubsystem txnSubsystem, TransactionContext context, int datasetId,
+    public void generateLogRecord(ITransactionSubsystem txnSubsystem, ITransactionContext context, int datasetId,
             int PKHashValue, long resourceId, IndexOperation newOperation, ITupleReference newValue,
             IndexOperation oldOperation, ITupleReference oldValue) throws ACIDException {
 
@@ -120,7 +123,7 @@
     }
 
     @Override
-    public void log(TransactionContext context, LogicalLogLocator logicalLogLocator, int logContentSize,
+    public void log(ITransactionContext context, LogicalLogLocator logicalLogLocator, int logContentSize,
             ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         int offset = 0;
         int tupleSize = 0;
@@ -175,12 +178,12 @@
     }
 
     @Override
-    public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject)
+    public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
             throws ACIDException {
     }
 
     @Override
-    public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject)
+    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
             throws ACIDException {
     }
 
@@ -207,67 +210,4 @@
         }
     }
 
-    /**
-     * Represents the state of a transaction thread. The state contains information
-     * that includes the tuple being operated, the operation and the location of the
-     * log record corresponding to the operation.
-     */
-    public class ReusableLogContentObject {
-
-        private LogicalLogLocator logicalLogLocator;
-        private IndexOperation newOperation;
-        private ITupleReference newValue;
-        private IndexOperation oldOperation;
-        private ITupleReference oldValue;
-
-        public ReusableLogContentObject(LogicalLogLocator logicalLogLocator, IndexOperation newOperation,
-                ITupleReference newValue, IndexOperation oldOperation, ITupleReference oldValue) {
-            this.logicalLogLocator = logicalLogLocator;
-            this.newOperation = newOperation;
-            this.newValue = newValue;
-            this.oldOperation = oldOperation;
-            this.oldValue = oldValue;
-        }
-
-        public synchronized LogicalLogLocator getLogicalLogLocator() {
-            return logicalLogLocator;
-        }
-
-        public synchronized void setLogicalLogLocator(LogicalLogLocator logicalLogLocator) {
-            this.logicalLogLocator = logicalLogLocator;
-        }
-
-        public synchronized void setNewOperation(IndexOperation newOperation) {
-            this.newOperation = newOperation;
-        }
-
-        public synchronized IndexOperation getNewOperation() {
-            return newOperation;
-        }
-
-        public synchronized void setNewValue(ITupleReference newValue) {
-            this.newValue = newValue;
-        }
-
-        public synchronized ITupleReference getNewValue() {
-            return newValue;
-        }
-
-        public synchronized void setOldOperation(IndexOperation oldOperation) {
-            this.oldOperation = oldOperation;
-        }
-
-        public synchronized IndexOperation getOldOperation() {
-            return oldOperation;
-        }
-
-        public synchronized void setOldValue(ITupleReference oldValue) {
-            this.oldValue = oldValue;
-        }
-
-        public synchronized ITupleReference getOldValue() {
-            return oldValue;
-        }
-    }
-
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
index da5042c..bd7d159 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
@@ -17,28 +17,38 @@
 import java.util.HashMap;
 import java.util.Map;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.MutableResourceId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.ILoggerRepository;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.MutableResourceId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 
-public class IndexLoggerRepository {
+public class IndexLoggerRepository implements ILoggerRepository {
 
-    private final Map<MutableResourceId, IndexLogger> loggers = new HashMap<MutableResourceId, IndexLogger>();
-    private final TransactionSubsystem txnSubsystem;
+    private final Map<MutableResourceId, ILogger> loggers = new HashMap<MutableResourceId, ILogger>();
+    private final ITransactionSubsystem txnSubsystem;
     private MutableResourceId mutableResourceId;
 
-    public IndexLoggerRepository(TransactionSubsystem provider) {
+    public IndexLoggerRepository(ITransactionSubsystem provider) {
         this.txnSubsystem = provider;
         mutableResourceId = new MutableResourceId(0);
     }
 
-    public synchronized IndexLogger getIndexLogger(long resourceId, byte resourceType) {
+    @Override
+    public synchronized ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException {
         mutableResourceId.setId(resourceId);
-        IndexLogger logger = loggers.get(mutableResourceId);
+        ILogger logger = loggers.get(mutableResourceId);
         if (logger == null) {
             MutableResourceId newMutableResourceId = new MutableResourceId(resourceId);
-            IIndex index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                    .getIndex(resourceId);
+            IIndex index;
+            try {
+                index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                        .getIndex(resourceId);
+            } catch (HyracksDataException e) {
+                throw new ACIDException(e);
+            }
             logger = new IndexLogger(resourceId, resourceType, index);
             loggers.put(newMutableResourceId, logger);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
index 7ecfa69..8506df7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
@@ -14,9 +14,12 @@
  */
 package edu.uci.ics.asterix.transaction.management.service.logging;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.IResourceManager;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -28,9 +31,9 @@
 
     public final byte resourceType;
 
-    private final TransactionSubsystem txnSubsystem;
+    private final ITransactionSubsystem txnSubsystem;
 
-    public IndexResourceManager(byte resourceType, TransactionSubsystem provider) {
+    public IndexResourceManager(byte resourceType, ITransactionSubsystem provider) {
         this.resourceType = resourceType;
         this.txnSubsystem = provider;
     }
@@ -43,11 +46,13 @@
         long resourceId = logRecordHelper.getResourceId(logLocator);
         int offset = logRecordHelper.getLogContentBeginPos(logLocator);
 
-        //TODO
-        //replace TransactionResourceRepository with IndexLifeCycleManager
-        // look up the repository to obtain the resource object
-        IIndex index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                .getIndex(resourceId);
+        IIndex index;
+        try {
+            index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                    .getIndex(resourceId);
+        } catch (HyracksDataException e1) {
+            throw new ACIDException("Cannot undo: unable to find index");
+        }
 
         /* field count */
         int fieldCount = logLocator.getBuffer().readInt(offset);
@@ -113,8 +118,13 @@
         long resourceId = logRecordHelper.getResourceId(logLocator);
         int offset = logRecordHelper.getLogContentBeginPos(logLocator);
 
-        IIndex index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                .getIndex(resourceId);
+        IIndex index;
+        try {
+            index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                    .getIndex(resourceId);
+        } catch (HyracksDataException e1) {
+            throw new ACIDException("Cannot redo: unable to find index");
+        }
 
         /* field count */
         int fieldCount = logLocator.getBuffer().readInt(offset);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
index 7e954d8..f452631 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
@@ -17,7 +17,15 @@
 import java.io.File;
 import java.io.IOException;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.FileUtil;
+import edu.uci.ics.asterix.common.transactions.IBuffer;
+import edu.uci.ics.asterix.common.transactions.IFileBasedBuffer;
+import edu.uci.ics.asterix.common.transactions.ILogCursor;
+import edu.uci.ics.asterix.common.transactions.ILogFilter;
+import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
 
 public class LogCursor implements ILogCursor {
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index 38d39cc..ba7e093 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -17,6 +17,7 @@
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
@@ -27,18 +28,32 @@
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.FileBasedBuffer;
+import edu.uci.ics.asterix.common.transactions.FileUtil;
+import edu.uci.ics.asterix.common.transactions.IFileBasedBuffer;
+import edu.uci.ics.asterix.common.transactions.ILogCursor;
+import edu.uci.ics.asterix.common.transactions.ILogFilter;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
+import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
 
-public class LogManager implements ILogManager {
+public class LogManager implements ILogManager, ILifeCycleComponent {
 
     public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(LogManager.class.getName());
@@ -78,7 +93,7 @@
      */
     private AtomicLong lsn = new AtomicLong(0);
 
-    private List<HashMap<TransactionContext, Integer>> activeTxnCountMaps;
+    private List<HashMap<ITransactionContext, Integer>> activeTxnCountMaps;
 
     public void addFlushRequest(int pageIndex, long lsn, boolean isSynchronous) {
         logPageFlusher.requestFlush(pageIndex, lsn, isSynchronous);
@@ -98,7 +113,7 @@
 
     public LogManager(TransactionSubsystem provider) throws ACIDException {
         this.provider = provider;
-        initLogManagerProperties(this.provider.getId());
+        logManagerProperties = new LogManagerProperties(this.provider.getTransactionProperties(), this.provider.getId());
         logPageSize = logManagerProperties.getLogPageSize();
         initLogManager();
         statLogSize = 0;
@@ -107,49 +122,20 @@
 
     public LogManager(TransactionSubsystem provider, String nodeId) throws ACIDException {
         this.provider = provider;
-        initLogManagerProperties(nodeId);
+        logManagerProperties = new LogManagerProperties(provider.getTransactionProperties(), nodeId);
         logPageSize = logManagerProperties.getLogPageSize();
         initLogManager();
         statLogSize = 0;
         statLogCount = 0;
     }
 
-    /*
-     * initialize the log manager properties either from the configuration file
-     * on disk or with default values
-     */
-    private void initLogManagerProperties(String nodeId) throws ACIDException {
-        LogManagerProperties logProperties = null;
-        InputStream is = null;
-        try {
-            is = this.getClass().getClassLoader()
-                    .getResourceAsStream(TransactionManagementConstants.LogManagerConstants.LOG_CONF_FILE);
-
-            Properties p = new Properties();
-
-            if (is != null) {
-                p.load(is);
-            }
-            logProperties = new LogManagerProperties(p, nodeId);
-
-        } catch (IOException ioe) {
-            if (is != null) {
-                try {
-                    is.close();
-                } catch (IOException e) {
-                    throw new ACIDException("unable to close input stream ", e);
-                }
-            }
-        }
-        logManagerProperties = logProperties;
-    }
-
     private void initLogManager() throws ACIDException {
         logRecordHelper = new LogRecordHelper(this);
         numLogPages = logManagerProperties.getNumLogPages();
-        activeTxnCountMaps = new ArrayList<HashMap<TransactionContext, Integer>>(numLogPages);
+        activeTxnCountMaps = new ArrayList<HashMap<ITransactionContext, Integer>>(numLogPages);
+
         for (int i = 0; i < numLogPages; i++) {
-            activeTxnCountMaps.add(new HashMap<TransactionContext, Integer>());
+            activeTxnCountMaps.add(new HashMap<ITransactionContext, Integer>());
         }
 
         logPages = new FileBasedBuffer[numLogPages];
@@ -172,7 +158,7 @@
          */
         logPageFlusher = new LogPageFlushThread(this);
         logPageFlusher.setDaemon(true);
-        logPageFlusher.start();
+        AsterixThreadExecutor.INSTANCE.execute(logPageFlusher);
     }
 
     public int getLogPageIndex(long lsnValue) {
@@ -264,10 +250,10 @@
                 // space in the log page and hence is an owner.
                 logPages[prevPage].incRefCnt();
                 logPages[prevPage].releaseReadLatch();
-                
+
                 // forward the nextWriteOffset in the log page
                 logPages[prevPage].setBufferNextWriteOffset(logPageSize);
-                
+
                 logPages[prevPage].decRefCnt();
 
                 addFlushRequest(prevPage, old, false);
@@ -298,11 +284,11 @@
     }
 
     @Override
-    public void log(byte logType, TransactionContext txnCtx, int datasetId, int PKHashValue, long resourceId,
+    public void log(byte logType, ITransactionContext txnCtx, int datasetId, int PKHashValue, long resourceId,
             byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
             LogicalLogLocator logicalLogLocator) throws ACIDException {
 
-        HashMap<TransactionContext, Integer> map = null;
+        HashMap<ITransactionContext, Integer> map = null;
         int activeTxnCount;
 
         // logLocator is a re-usable object that is appropriately set in each
@@ -412,6 +398,11 @@
             logPages[pageIndex].setBufferNextWriteOffset(bufferNextWriteOffset);
 
             if (logType != LogType.ENTITY_COMMIT) {
+                if (logType == LogType.COMMIT) {
+                    txnCtx.setExclusiveJobLevelCommit();
+                    map = activeTxnCountMaps.get(pageIndex);
+                    map.put(txnCtx, 1);
+                }
                 // release the ownership as the log record has been placed in
                 // created space.
                 logPages[pageIndex].decRefCnt();
@@ -720,15 +711,17 @@
         return provider;
     }
 
+    static AtomicInteger t = new AtomicInteger();
+
     public void decrementActiveTxnCountOnIndexes(int pageIndex) throws HyracksDataException {
-        TransactionContext ctx = null;
+        ITransactionContext ctx = null;
         int count = 0;
         int i = 0;
 
-        HashMap<TransactionContext, Integer> map = activeTxnCountMaps.get(pageIndex);
-        Set<Map.Entry<TransactionContext, Integer>> entrySet = map.entrySet();
+        HashMap<ITransactionContext, Integer> map = activeTxnCountMaps.get(pageIndex);
+        Set<Map.Entry<ITransactionContext, Integer>> entrySet = map.entrySet();
         if (entrySet != null) {
-            for (Map.Entry<TransactionContext, Integer> entry : entrySet) {
+            for (Map.Entry<ITransactionContext, Integer> entry : entrySet) {
                 if (entry != null) {
                     if (entry.getValue() != null) {
                         count = entry.getValue();
@@ -745,6 +738,60 @@
 
         map.clear();
     }
+
+    @Override
+    public void start() {
+        //no op
+    }
+
+    @Override
+    public void stop(boolean dumpState, OutputStream os) {
+        if (dumpState) {
+            //#. dump Configurable Variables
+            dumpConfVars(os);
+
+            //#. dump LSNInfo
+            dumpLSNInfo(os);
+
+            try {
+                os.flush();
+            } catch (IOException e) {
+                //ignore
+            }
+        }
+    }
+
+    private void dumpConfVars(OutputStream os) {
+        try {
+            StringBuilder sb = new StringBuilder();
+            sb.append("\n>>dump_begin\t>>----- [ConfVars] -----");
+            sb.append(logManagerProperties.toString());
+            sb.append("\n>>dump_end\t>>----- [ConfVars] -----\n");
+            os.write(sb.toString().getBytes());
+        } catch (Exception e) {
+            //ignore exception and continue dumping as much as possible.
+            if (IS_DEBUG_MODE) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    private void dumpLSNInfo(OutputStream os) {
+        try {
+            StringBuilder sb = new StringBuilder();
+            sb.append("\n>>dump_begin\t>>----- [LSNInfo] -----");
+            sb.append("\nstartingLSN: " + startingLSN);
+            sb.append("\ncurrentLSN: " + lsn.get());
+            sb.append("\nlastFlushedLSN: " + lastFlushedLSN.get());
+            sb.append("\n>>dump_end\t>>----- [LSNInfo] -----\n");
+            os.write(sb.toString().getBytes());
+        } catch (Exception e) {
+            //ignore exception and continue dumping as much as possible.
+            if (IS_DEBUG_MODE) {
+                e.printStackTrace();
+            }
+        }
+    }
 }
 
 /*
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
index 1b65d8f..ed30b88 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
@@ -14,7 +14,12 @@
  */
 package edu.uci.ics.asterix.transaction.management.service.logging;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
+import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
 
 /**
  * An implementation of the @see ILogRecordHelper interface that provides API
@@ -63,9 +68,7 @@
     private final int RESOURCE_ID_POS = 25;
     private final int RESOURCE_MGR_ID_POS = 33;
     private final int LOG_RECORD_SIZE_POS = 34;
-    
 
-    
     private ILogManager logManager;
 
     public LogRecordHelper(ILogManager logManager) {
@@ -181,7 +184,7 @@
     }
 
     @Override
-    public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, TransactionContext context,
+    public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
             int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
             int logRecordSize) {
 
@@ -258,8 +261,8 @@
     public int getLogChecksumSize() {
         return LOG_CHECKSUM_SIZE;
     }
-    
+
     public int getCommitLogSize() {
         return COMMIT_LOG_SIZE;
     }
-}
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogUtil.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogUtil.java
index 77a99bc..e57bf32 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogUtil.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogUtil.java
@@ -24,7 +24,13 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.FileUtil;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
+import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
+
 
 /**
  * A utility class providing helper methods for the {@link ILogManager}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
index 3eb87bc..68df619 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
@@ -2,16 +2,17 @@
 
 import java.util.List;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 
 public class CheckpointThread extends Thread {
 
-    private static final long LSN_THRESHOLD = 64 * 1024 * 1024;
-    private long checkpointTermInSecs = 120; //seconds.
+    private long lsnThreshold;
+    private long checkpointTermInSecs;
 
     private long lastMinMCTFirstLSN = 0;
 
@@ -19,12 +20,11 @@
     private final IIndexLifecycleManager indexLifecycleManager;
 
     public CheckpointThread(IRecoveryManager recoveryMgr, IIndexLifecycleManager indexLifecycleManager,
-            long checkpointTermInSecs) {
+            long lsnThreshold, long checkpointTermInSecs) {
         this.recoveryMgr = recoveryMgr;
         this.indexLifecycleManager = indexLifecycleManager;
-        if (this.checkpointTermInSecs < checkpointTermInSecs) {
-            this.checkpointTermInSecs = checkpointTermInSecs;
-        }
+        this.lsnThreshold = lsnThreshold;
+        this.checkpointTermInSecs = checkpointTermInSecs;
     }
 
     @Override
@@ -38,7 +38,7 @@
             }
 
             currentMinMCTFirstLSN = getMinMCTFirstLSN();
-            if (currentMinMCTFirstLSN - lastMinMCTFirstLSN > LSN_THRESHOLD) {
+            if (currentMinMCTFirstLSN - lastMinMCTFirstLSN > lsnThreshold) {
                 try {
                     recoveryMgr.checkpoint(false);
                     lastMinMCTFirstLSN = currentMinMCTFirstLSN;
@@ -55,7 +55,7 @@
         long firstLSN;
         if (openIndexList.size() > 0) {
             for (IIndex index : openIndexList) {
-                firstLSN = ((IndexOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
+                firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
                 minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
             }
         } else {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
index 47c5dcb..7b95358 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -22,7 +22,9 @@
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
+import java.io.OutputStream;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -34,27 +36,30 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
-import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
-import edu.uci.ics.asterix.transaction.management.service.logging.IBuffer;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogCursor;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogFilter;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogRecordHelper;
+import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.asterix.common.transactions.IBuffer;
+import edu.uci.ics.asterix.common.transactions.ILogCursor;
+import edu.uci.ics.asterix.common.transactions.ILogFilter;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
+import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
+import edu.uci.ics.asterix.common.transactions.IResourceManager;
+import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.logging.PhysicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManager;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
@@ -81,11 +86,12 @@
  * not in place completely. Once we have physical logging implemented, we would
  * add support for crash recovery.
  */
-public class RecoveryManager implements IRecoveryManager {
+public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
 
     public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(RecoveryManager.class.getName());
     private final TransactionSubsystem txnSubsystem;
+    private final int checkpointHistory;
 
     /**
      * A file at a known location that contains the LSN of the last log record
@@ -96,6 +102,7 @@
 
     public RecoveryManager(TransactionSubsystem TransactionProvider) throws ACIDException {
         this.txnSubsystem = TransactionProvider;
+        this.checkpointHistory = this.txnSubsystem.getTransactionProperties().getCheckpointHistory();
     }
 
     /**
@@ -428,7 +435,7 @@
         if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting sharp checkpoint ... ");
         }
-        
+
         LogManager logMgr = (LogManager) txnSubsystem.getLogManager();
         TransactionManager txnMgr = (TransactionManager) txnSubsystem.getTransactionManager();
         String logDir = logMgr.getLogManagerProperties().getLogDir();
@@ -448,7 +455,7 @@
                 ILSMIndex lsmIndex = (ILSMIndex) index;
                 ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(NoOpOperationCallback.INSTANCE,
                         NoOpOperationCallback.INSTANCE);
-                IndexOperationTracker indexOpTracker = (IndexOperationTracker) lsmIndex.getOperationTracker();
+                BaseOperationTracker indexOpTracker = (BaseOperationTracker) lsmIndex.getOperationTracker();
                 BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
                         indexOpTracker.getIOOperationCallback());
                 callbackList.add(cb);
@@ -473,7 +480,7 @@
         long firstLSN;
         if (openIndexList.size() > 0) {
             for (IIndex index : openIndexList) {
-                firstLSN = ((IndexOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
+                firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
                 minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
             }
         } else {
@@ -512,15 +519,17 @@
 
         //#. delete the previous checkpoint files
         if (prevCheckpointFiles != null) {
-            for (File file : prevCheckpointFiles) {
-                file.delete();
+            // sort the filenames lexicographically to keep the latest checkpointHistory files.
+            Arrays.sort(prevCheckpointFiles);
+            for (int i = 0; i < prevCheckpointFiles.length - this.checkpointHistory; ++i) {
+                prevCheckpointFiles[i].delete();
             }
         }
 
         if (isSharpCheckpoint) {
             logMgr.renewLogFiles();
         }
-        
+
         if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Completed sharp checkpoint.");
         }
@@ -611,7 +620,7 @@
      * @see edu.uci.ics.transaction.management.service.recovery.IRecoveryManager# rollbackTransaction (edu.uci.ics.TransactionContext.management.service.transaction .TransactionContext)
      */
     @Override
-    public void rollbackTransaction(TransactionContext txnContext) throws ACIDException {
+    public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException {
         ILogManager logManager = txnSubsystem.getLogManager();
         ILogRecordHelper logRecordHelper = logManager.getLogRecordHelper();
         Map<TxnId, List<Long>> loserTxnTable = new HashMap<TxnId, List<Long>>();
@@ -780,6 +789,16 @@
                     + undoCount);
         }
     }
+
+    @Override
+    public void start() {
+        //no op
+    }
+
+    @Override
+    public void stop(boolean dumpState, OutputStream os) {
+        //no op
+    }
 }
 
 class TxnId {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
index 9bfcc8f..15e21bc 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
@@ -14,8 +14,8 @@
  */
 package edu.uci.ics.asterix.transaction.management.service.recovery;
 
-import edu.uci.ics.asterix.transaction.management.service.logging.PhysicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
+import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
 
 /**
  * Represents a bookkeeping data-structure that is populated duing the analysis
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
new file mode 100644
index 0000000..1a9716f
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
@@ -0,0 +1,104 @@
+package edu.uci.ics.asterix.transaction.management.service.transaction;
+
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+
+public class AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
+        ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider, ILSMOperationTrackerProvider,
+        ILSMIOOperationCallbackProvider {
+    private static final long serialVersionUID = 1L;
+
+    private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+    private final boolean isSecondary;
+
+    public static final AsterixRuntimeComponentsProvider LSMBTREE_PRIMARY_PROVIDER = new AsterixRuntimeComponentsProvider(
+            LSMBTreeIOOperationCallbackFactory.INSTANCE, false);
+    public static final AsterixRuntimeComponentsProvider LSMBTREE_SECONDARY_PROVIDER = new AsterixRuntimeComponentsProvider(
+            LSMBTreeIOOperationCallbackFactory.INSTANCE, true);
+    public static final AsterixRuntimeComponentsProvider LSMRTREE_PROVIDER = new AsterixRuntimeComponentsProvider(
+            LSMRTreeIOOperationCallbackFactory.INSTANCE, true);
+    public static final AsterixRuntimeComponentsProvider LSMINVERTEDINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(
+            LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, true);
+    public static final AsterixRuntimeComponentsProvider NOINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(null,
+            false);
+
+    private AsterixRuntimeComponentsProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean isSecondary) {
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
+        this.isSecondary = isSecondary;
+    }
+
+    @Override
+    public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
+        assert isSecondary;
+        return new BaseOperationTracker(ioOpCallbackFactory);
+    }
+
+    @Override
+    public ILSMIOOperationCallback getIOOperationCallback(ILSMIndex index) {
+        return ((BaseOperationTracker) index.getOperationTracker()).getIOOperationCallback();
+    }
+
+    @Override
+    public ILSMIOOperationScheduler getIOScheduler(IHyracksTaskContext ctx) {
+        return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getLSMIOScheduler();
+    }
+
+    @Override
+    public ILSMMergePolicy getMergePolicy(IHyracksTaskContext ctx) {
+        return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getLSMMergePolicy();
+    }
+
+    @Override
+    public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
+        return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getBufferCache();
+    }
+
+    @Override
+    public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
+        return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getFileMapManager();
+    }
+
+    @Override
+    public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
+        return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getLocalResourceRepository();
+    }
+
+    @Override
+    public IIndexLifecycleManager getLifecycleManager(IHyracksTaskContext ctx) {
+        return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getIndexLifecycleManager();
+    }
+
+    @Override
+    public ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
+        return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getResourceIdFactory();
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobIdFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobIdFactory.java
index cac4f8e..8f9fa79 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobIdFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobIdFactory.java
@@ -16,6 +16,8 @@
 
 import java.util.concurrent.atomic.AtomicInteger;
 
+import edu.uci.ics.asterix.common.transactions.JobId;
+
 /**
  * Represents a factory to generate unique transaction IDs.
  */
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
index 5b01edf..05ee899 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -15,19 +15,19 @@
 package edu.uci.ics.asterix.transaction.management.service.transaction;
 
 import java.io.Serializable;
-import java.util.ArrayList;
 import java.util.HashSet;
-import java.util.List;
+import java.util.Iterator;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.opcallbacks.AbstractOperationCallback;
-import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
-import edu.uci.ics.asterix.transaction.management.resource.ICloseable;
+import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.ICloseable;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager.TransactionState;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
@@ -40,17 +40,7 @@
  * initiate an operation on the behalf of the transaction associated with the
  * context.
  */
-public class TransactionContext implements Serializable {
-
-    public static final long INVALID_TIME = -1l; // used for showing a
-    // transaction is not waiting.
-    public static final int ACTIVE_STATUS = 0;
-    public static final int TIMED_OUT_STATUS = 1;
-
-    public enum TransactionType {
-        READ,
-        READ_WRITE
-    }
+public class TransactionContext implements ITransactionContext, Serializable {
 
     private static final long serialVersionUID = -6105616785783310111L;
     private TransactionSubsystem transactionSubsystem;
@@ -62,13 +52,14 @@
     private Set<ICloseable> resources = new HashSet<ICloseable>();
     private TransactionType transactionType = TransactionType.READ;
     private JobId jobId;
+    private boolean exlusiveJobLevelCommit;
 
     // List of indexes on which operations were performed on behalf of this transaction.
-    private final List<ILSMIndex> indexes = new ArrayList<ILSMIndex>();
+    private final Set<ILSMIndex> indexes = new HashSet<ILSMIndex>();
 
     // List of operation callbacks corresponding to the operand indexes. In particular, needed to track
     // the number of active operations contributed by this transaction.
-    private final List<AbstractOperationCallback> callbacks = new ArrayList<AbstractOperationCallback>();
+    private final Set<AbstractOperationCallback> callbacks = new HashSet<AbstractOperationCallback>();
 
     public TransactionContext(JobId jobId, TransactionSubsystem transactionSubsystem) throws ACIDException {
         this.jobId = jobId;
@@ -94,19 +85,44 @@
     public void updateLastLSNForIndexes(long lastLSN) {
         synchronized (indexes) {
             for (ILSMIndex index : indexes) {
-                ((IndexOperationTracker) index.getOperationTracker()).updateLastLSN(lastLSN);
+                ((BaseOperationTracker) index.getOperationTracker()).updateLastLSN(lastLSN);
             }
         }
     }
 
     public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException {
         synchronized (indexes) {
-            for (int i = 0; i < indexes.size(); i++) {
-                ILSMIndex index = indexes.get(i);
-                IModificationOperationCallback modificationCallback = (IModificationOperationCallback) callbacks.get(i);
-                ((IndexOperationTracker) index.getOperationTracker()).completeOperation(LSMOperationType.MODIFICATION,
-                        null, modificationCallback);
+            Set<BaseOperationTracker> opTrackers = new HashSet<BaseOperationTracker>();
+            Iterator<ILSMIndex> indexIt = indexes.iterator();
+            Iterator<AbstractOperationCallback> cbIt = callbacks.iterator();
+            while (indexIt.hasNext()) {
+                ILSMIndex index = indexIt.next();
+                opTrackers.add((BaseOperationTracker) index.getOperationTracker());
+                assert cbIt.hasNext();
             }
+            Iterator<BaseOperationTracker> trackerIt = opTrackers.iterator();
+            while (trackerIt.hasNext()) {
+                IModificationOperationCallback modificationCallback = (IModificationOperationCallback) cbIt.next();
+                BaseOperationTracker opTracker = (BaseOperationTracker) trackerIt.next();
+                if (exlusiveJobLevelCommit) {
+                    opTracker.exclusiveJobCommitted();
+                } else {
+                    opTracker.completeOperation(null, LSMOperationType.MODIFICATION, null, modificationCallback);
+                }
+            }
+        }
+    }
+
+    @Override
+    public int getActiveOperationCountOnIndexes() throws HyracksDataException {
+        synchronized (indexes) {
+            int count = 0;
+            Iterator<AbstractOperationCallback> cbIt = callbacks.iterator();
+            while (cbIt.hasNext()) {
+                IModificationOperationCallback modificationCallback = (IModificationOperationCallback) cbIt.next();
+                count += ((AbstractOperationCallback) modificationCallback).getLocalNumActiveOperations();
+            }
+            return count;
         }
     }
 
@@ -170,7 +186,7 @@
             closeable.close(this);
         }
     }
-    
+
     @Override
     public int hashCode() {
         return jobId.getId();
@@ -180,4 +196,21 @@
     public boolean equals(Object o) {
         return (o == this);
     }
+
+    @Override
+    public void setExclusiveJobLevelCommit() {
+        exlusiveJobLevelCommit = true;
+    }
+
+    public String prettyPrint() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("\n" + jobId + "\n");
+        sb.append("transactionType: " + transactionType);
+        sb.append("firstLogLocator: " + firstLogLocator.getLsn() + "\n");
+        sb.append("lastLogLocator: " + lastLogLocator.getLsn() + "\n");
+        sb.append("TransactionState: " + txnState + "\n");
+        sb.append("startWaitTime: " + startWaitTime + "\n");
+        sb.append("status: " + status + "\n");
+        return sb.toString();
+    }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
index 7d04f61..25b9195 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
@@ -26,10 +26,6 @@
     }
 
     public static class LogManagerConstants {
-        public static final String LOG_CONF_DIR = "log_conf";
-        public static final String LOG_CONF_FILE = "log.properties";
-        public static final String ASTERIX_CONF_DIR = "src/main/resources";
-        public static final String DEFAULT_LOG_DIR = "asterix_logs";
         public static final int TERMINAL_LSN = -1;
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
index d3294d0..09fa028 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
@@ -14,24 +14,34 @@
  */
 package edu.uci.ics.asterix.transaction.management.service.transaction;
 
+import java.io.IOException;
+import java.io.OutputStream;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
+import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
 
 /**
  * An implementation of the @see ITransactionManager interface that provides
  * implementation of APIs for governing the lifecycle of a transaction.
  */
-public class TransactionManager implements ITransactionManager {
+public class TransactionManager implements ITransactionManager, ILifeCycleComponent {
+
+    public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(TransactionManager.class.getName());
     private final TransactionSubsystem transactionProvider;
-    private Map<JobId, TransactionContext> transactionContextRepository = new HashMap<JobId, TransactionContext>();
+    private Map<JobId, ITransactionContext> transactionContextRepository = new HashMap<JobId, ITransactionContext>();
     private AtomicInteger maxJobId = new AtomicInteger(0);
 
     public TransactionManager(TransactionSubsystem provider) {
@@ -39,7 +49,7 @@
     }
 
     @Override
-    public void abortTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal)
+    public void abortTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
             throws ACIDException {
         synchronized (txnContext) {
             if (txnContext.getTxnState().equals(TransactionState.ABORTED)) {
@@ -65,9 +75,9 @@
     }
 
     @Override
-    public TransactionContext beginTransaction(JobId jobId) throws ACIDException {
+    public ITransactionContext beginTransaction(JobId jobId) throws ACIDException {
         setMaxJobId(jobId.getId());
-        TransactionContext txnContext = new TransactionContext(jobId, transactionProvider);
+        ITransactionContext txnContext = new TransactionContext(jobId, transactionProvider);
         synchronized (this) {
             transactionContextRepository.put(jobId, txnContext);
         }
@@ -75,11 +85,11 @@
     }
 
     @Override
-    public TransactionContext getTransactionContext(JobId jobId) throws ACIDException {
+    public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException {
         setMaxJobId(jobId.getId());
         synchronized (transactionContextRepository) {
 
-            TransactionContext context = transactionContextRepository.get(jobId);
+            ITransactionContext context = transactionContextRepository.get(jobId);
             if (context == null) {
                 context = transactionContextRepository.get(jobId);
                 context = new TransactionContext(jobId, transactionProvider);
@@ -90,7 +100,7 @@
     }
 
     @Override
-    public void commitTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal)
+    public void commitTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
             throws ACIDException {
         synchronized (txnContext) {
             if ((txnContext.getTxnState().equals(TransactionState.COMMITTED))) {
@@ -102,21 +112,29 @@
 
             //for entity-level commit
             if (PKHashVal != -1) {
-                transactionProvider.getLockManager().unlock(datasetId, PKHashVal, txnContext, true);
-                /*****************************
-                try {
-                    //decrease the transaction reference count on index
-                    txnContext.decreaseActiveTransactionCountOnIndexes();
-                } catch (HyracksDataException e) {
-                    throw new ACIDException("failed to complete index operation", e);
+                boolean countIsZero = transactionProvider.getLockManager().unlock(datasetId, PKHashVal, txnContext,
+                        true);
+                if (!countIsZero) {
+                    // Lock count != 0 for a particular entity implies that the entity has been locked 
+                    // more than once (probably due to a hash collision in our current model).
+                    // It is safe to decrease the active transaction count on indexes since,  
+                    // by virtue of the counter not being zero, there is another transaction 
+                    // that has increased the transaction count. Thus, decreasing it will not 
+                    // allow the data to be flushed (yet). The flush will occur when the log page
+                    // flush thread decides to decrease the count for the last time.
+                    try {
+                        //decrease the transaction reference count on index
+                        txnContext.decreaseActiveTransactionCountOnIndexes();
+                    } catch (HyracksDataException e) {
+                        throw new ACIDException("failed to complete index operation", e);
+                    }
                 }
-                *****************************/
                 return;
             }
 
             //for job-level commit
             try {
-                if (txnContext.getTransactionType().equals(TransactionContext.TransactionType.READ_WRITE)) {
+                if (txnContext.getTransactionType().equals(ITransactionContext.TransactionType.READ_WRITE)) {
                     transactionProvider.getLogManager().log(LogType.COMMIT, txnContext, -1, -1, -1, (byte) 0, 0, null,
                             null, txnContext.getLastLogLocator());
                 }
@@ -135,7 +153,7 @@
     }
 
     @Override
-    public void completedTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal, boolean success)
+    public void completedTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal, boolean success)
             throws ACIDException {
         if (!success) {
             abortTransaction(txnContext, datasetId, PKHashVal);
@@ -148,12 +166,69 @@
     public TransactionSubsystem getTransactionProvider() {
         return transactionProvider;
     }
-    
+
     public void setMaxJobId(int jobId) {
         maxJobId.set(Math.max(maxJobId.get(), jobId));
     }
-    
+
     public int getMaxJobId() {
         return maxJobId.get();
     }
+
+    @Override
+    public void start() {
+        //no op
+    }
+
+    @Override
+    public void stop(boolean dumpState, OutputStream os) {
+        if (dumpState) {
+            //#. dump TxnContext
+            dumpTxnContext(os);
+
+            try {
+                os.flush();
+            } catch (IOException e) {
+                //ignore
+            }
+        }
+    }
+
+    private void dumpTxnContext(OutputStream os) {
+        JobId jobId;
+        ITransactionContext txnCtx;
+        StringBuilder sb = new StringBuilder();
+
+        try {
+            sb.append("\n>>dump_begin\t>>----- [ConfVars] -----");
+            Set<Map.Entry<JobId, ITransactionContext>> entrySet = transactionContextRepository.entrySet();
+            if (entrySet != null) {
+                for (Map.Entry<JobId, ITransactionContext> entry : entrySet) {
+                    if (entry != null) {
+                        jobId = entry.getKey();
+                        if (jobId != null) {
+                            sb.append("\n" + jobId);
+                        } else {
+                            sb.append("\nJID:null");
+                        }
+
+                        txnCtx = entry.getValue();
+                        if (txnCtx != null) {
+                            sb.append(txnCtx.prettyPrint());
+                        } else {
+                            sb.append("\nTxnCtx:null");
+                        }
+                    }
+                }
+            }
+
+            sb.append("\n>>dump_end\t>>----- [ConfVars] -----\n");
+            os.write(sb.toString().getBytes());
+        } catch (Exception e) {
+            //ignore exception and continue dumping as much as possible.
+            if (IS_DEBUG_MODE) {
+                e.printStackTrace();
+            }
+        }
+    }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
index c83b7bc..bde44de 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
@@ -14,23 +14,26 @@
  */
 package edu.uci.ics.asterix.transaction.management.service.transaction;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceManagerRepository;
-import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
 import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.IndexLoggerRepository;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
 import edu.uci.ics.asterix.transaction.management.service.recovery.CheckpointThread;
-import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
-import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager;
 import edu.uci.ics.asterix.transaction.management.service.recovery.RecoveryManager;
 
 /**
  * Provider for all the sub-systems (transaction/lock/log/recovery) managers.
  * Users of transaction sub-systems must obtain them from the provider.
  */
-public class TransactionSubsystem {
+public class TransactionSubsystem implements ITransactionSubsystem {
     private final String id;
     private final ILogManager logManager;
     private final ILockManager lockManager;
@@ -40,10 +43,12 @@
     private final IndexLoggerRepository loggerRepository;
     private final IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider;
     private final CheckpointThread checkpointThread;
+    private final AsterixTransactionProperties txnProperties;
 
-    public TransactionSubsystem(String id, IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider)
-            throws ACIDException {
+    public TransactionSubsystem(String id, IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider,
+            AsterixTransactionProperties txnProperties) throws ACIDException {
         this.id = id;
+        this.txnProperties = txnProperties;
         this.transactionManager = new TransactionManager(this);
         this.logManager = new LogManager(this);
         this.lockManager = new LockManager(this);
@@ -52,10 +57,11 @@
         this.resourceRepository = new TransactionalResourceManagerRepository();
         this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
         if (asterixAppRuntimeContextProvider != null) {
-	        this.checkpointThread = new CheckpointThread(recoveryManager,
-	                asterixAppRuntimeContextProvider.getIndexLifecycleManager(), 0);
+            this.checkpointThread = new CheckpointThread(recoveryManager,
+                    asterixAppRuntimeContextProvider.getIndexLifecycleManager(),
+                    this.txnProperties.getCheckpointLSNThreshold(), this.txnProperties.getCheckpointPollFrequency());
         } else {
-        	this.checkpointThread = null;
+            this.checkpointThread = null;
         }
     }
 
@@ -87,6 +93,10 @@
         return asterixAppRuntimeContextProvider;
     }
 
+    public AsterixTransactionProperties getTransactionProperties() {
+        return txnProperties;
+    }
+
     public String getId() {
         return id;
     }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystemProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystemProvider.java
new file mode 100644
index 0000000..ab89ef4
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystemProvider.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.transaction;
+
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * The purpose of this provider is to work around a cyclic dependency between asterix-common and asterix-transactions.
+ * The operation callbacks would depend on the AsterixAppRuntimeContext to get the transaction subsystem,
+ * while at the same time the AsterixAppRuntimeContext depends on asterix-transactions for the TransactionSubsystem.
+ */
+public class TransactionSubsystemProvider implements ITransactionSubsystemProvider {
+    @Override
+    public ITransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx) {
+        IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+                .getApplicationContext().getApplicationObject();
+        return runtimeCtx.getTransactionSubsystem();
+    }
+}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
index c9d01a0..076227b 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
@@ -16,19 +16,19 @@
 
 import java.util.Random;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.logging.IBuffer;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogger;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.IBuffer;
+import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
 
 public class BasicLogger implements ILogger {
 
     private static long averageContentCreationTime = 0;
     private static long count = 0;
 
-    public void log(TransactionContext context, LogicalLogLocator wMemLSN, int length,
+    public void log(ITransactionContext context, LogicalLogLocator wMemLSN, int length,
             ReusableLogContentObject reusableLogContentObject) throws ACIDException {
 
         byte[] logContent = getRandomBytes(length);
@@ -66,12 +66,12 @@
         return averageContentCreationTime;
     }
 
-    public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
+    public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         // TODO Auto-generated method stub
 
     }
 
-    public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
+    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         // TODO Auto-generated method stub
 
     }
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
index fbc44da..a0e0980 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.asterix.transaction.management.logging;
 
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogger;
+import edu.uci.ics.asterix.common.transactions.ILogger;
 
 public interface IResource {
 
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
index 54ec036..862c996 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
@@ -16,8 +16,11 @@
 
 import java.io.IOException;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager;
+import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 public class RecoverySimulator {
@@ -28,10 +31,11 @@
         recoveryManager.startRecovery(true);
     }
 
-    public static void main(String args[]) throws IOException, ACIDException {
+    public static void main(String args[]) throws IOException, ACIDException, AsterixException {
         String id = "nc1";
         try {
-            TransactionSubsystem factory = new TransactionSubsystem(id, null);
+            TransactionSubsystem factory = new TransactionSubsystem(id, null, new AsterixTransactionProperties(
+                    new AsterixPropertiesAccessor()));
             IRecoveryManager recoveryManager = factory.getRecoveryManager();
             recoveryManager.startRecovery(true);
         } catch (ACIDException acide) {
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
index dcd644d..b9605ef 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
@@ -18,18 +18,21 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
+import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.IResourceManager;
+import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.logging.BasicLogger;
-import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogRecordHelper;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogActionType;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
 import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
@@ -48,8 +51,9 @@
         transactions = new Transaction[workload.numActiveThreads];
     }
 
-    public void beginWorkload() throws ACIDException {
-        provider = new TransactionSubsystem("nc1", null);
+    public void beginWorkload() throws ACIDException, AsterixException {
+        provider = new TransactionSubsystem("nc1", null, new AsterixTransactionProperties(
+                new AsterixPropertiesAccessor()));
         logManager = provider.getLogManager();
         lockManager = provider.getLockManager();
         provider.getTransactionalResourceRepository().registerTransactionalResourceManager(DummyResourceMgr.id,
@@ -83,7 +87,7 @@
         System.out.println(" Avg Content Creation time :" + BasicLogger.getAverageContentCreationTime());
     }
 
-    public static void main(String args[]) {
+    public static void main(String args[]) throws AsterixException {
         WorkloadProperties workload = new WorkloadProperties();
         TransactionWorkloadSimulator simulator = new TransactionWorkloadSimulator(workload);
         try {
@@ -177,12 +181,12 @@
                 byte logActionType = LogActionType.REDO_UNDO;
                 long pageId = 0;
                 if (!retry) {
-                    lockMode = (byte)(random.nextInt(2));
+                    lockMode = (byte) (random.nextInt(2));
                 }
                 tempDatasetId.setId(resourceID);
                 TransactionWorkloadSimulator.lockManager.lock(tempDatasetId, -1, lockMode, context);
-                TransactionWorkloadSimulator.logManager.log(logType, context, resourceID,
-                        -1, resourceID, ResourceType.LSM_BTREE, logSize, null, logger, memLSN);
+                TransactionWorkloadSimulator.logManager.log(logType, context, resourceID, -1, resourceID,
+                        ResourceType.LSM_BTREE, logSize, null, logger, memLSN);
                 retry = false;
                 Thread.currentThread().sleep(TransactionWorkloadSimulator.workload.thinkTime);
                 logCount.incrementAndGet();
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
index 54bb0b8..3b251bc 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
@@ -15,12 +15,12 @@
 package edu.uci.ics.asterix.transaction.management.test;
 
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
 import edu.uci.ics.asterix.transaction.management.logging.IResource;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogger;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 
 class FileLogger implements ILogger {
 
@@ -39,13 +39,13 @@
     }
 
     @Override
-    public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
+    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         // TODO Auto-generated method stub
 
     }
 
     @Override
-    public void log(TransactionContext context, final LogicalLogLocator memLSN, int logContentSize,
+    public void log(ITransactionContext context, final LogicalLogLocator memLSN, int logContentSize,
             ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         byte[] buffer = memLSN.getBuffer().getArray();
         byte[] content = logRecordContent.getBytes();
@@ -58,7 +58,7 @@
     }
 
     @Override
-    public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
+    public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         // TODO Auto-generated method stub
 
     }
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
index bc1a6f0..fc1eb86 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
@@ -21,9 +21,9 @@
 import java.io.FileWriter;
 import java.io.IOException;
 
+import edu.uci.ics.asterix.common.transactions.FileUtil;
+import edu.uci.ics.asterix.common.transactions.ILogger;
 import edu.uci.ics.asterix.transaction.management.logging.IResource;
-import edu.uci.ics.asterix.transaction.management.service.logging.FileUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogger;
 
 class FileResource implements IResource {
 
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
index 6014003..f47b1df 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
@@ -17,11 +17,11 @@
 import java.util.HashMap;
 import java.util.Map;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.IResourceManager;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.logging.IResource;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogRecordHelper;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
 
 class FileResourceManager implements IResourceManager {
 
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
index ad1238a..1aa8454 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
@@ -15,19 +15,17 @@
 package edu.uci.ics.asterix.transaction.management.test;
 
 import java.io.IOException;
-import java.util.Properties;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.logging.IBuffer;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogCursor;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogFilter;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogRecordHelper;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.IBuffer;
+import edu.uci.ics.asterix.common.transactions.ILogCursor;
+import edu.uci.ics.asterix.common.transactions.ILogFilter;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
+import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogManagerProperties;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.logging.PhysicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 public class LogRecordReader {
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
index 2bd277c..ccd83eb 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.asterix.transaction.management.test;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
 
 public class TransactionRecoverySimulator {
 
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
index 092f2ed..c322b31 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
@@ -17,23 +17,25 @@
 import java.io.IOException;
 import java.util.Random;
 
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
+import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
+import edu.uci.ics.asterix.common.transactions.IResourceManager;
+import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
+import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.logging.IResource;
-import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.ILogger;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager;
-import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager.SystemState;
-import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 public class TransactionSimulator {
@@ -48,9 +50,10 @@
     private LogicalLogLocator memLSN;
     private TransactionSubsystem transactionProvider;
 
-    public TransactionSimulator(IResource resource, IResourceManager resourceMgr) throws ACIDException {
+    public TransactionSimulator(IResource resource, IResourceManager resourceMgr) throws ACIDException, AsterixException {
         String id = "nc1";
-        transactionProvider = new TransactionSubsystem(id, null);
+        transactionProvider = new TransactionSubsystem(id, null, new AsterixTransactionProperties(
+                new AsterixPropertiesAccessor()));
         transactionManager = transactionProvider.getTransactionManager();
         logManager = transactionProvider.getLogManager();
         lockManager = transactionProvider.getLockManager();
@@ -63,12 +66,12 @@
         memLSN = LogUtil.getDummyLogicalLogLocator(transactionProvider.getLogManager());
     }
 
-    public TransactionContext beginTransaction() throws ACIDException {
+    public ITransactionContext beginTransaction() throws ACIDException {
         JobId jobId = JobIdFactory.generateJobId();
         return transactionManager.beginTransaction(jobId);
     }
 
-    public void executeTransactionOperation(TransactionContext txnContext, FileResource.CounterOperation operation)
+    public void executeTransactionOperation(ITransactionContext txnContext, FileResource.CounterOperation operation)
             throws ACIDException {
         // lockManager.lock(txnContext, resourceId, 0);
         ILogManager logManager = transactionProvider.getLogManager();
@@ -91,7 +94,7 @@
 
     }
 
-    public void commitTransaction(TransactionContext context) throws ACIDException {
+    public void commitTransaction(ITransactionContext context) throws ACIDException {
         transactionManager.commitTransaction(context, new DatasetId(-1), -1);
     }
 
@@ -103,7 +106,7 @@
     /**
      * @param args
      */
-    public static void main(String[] args) throws IOException, ACIDException {
+    public static void main(String[] args) throws IOException, ACIDException, AsterixException {
         String fileDir = "testdata";
         String fileName = "counterFile";
         IResource resource = new FileResource(fileDir, fileName);
@@ -116,7 +119,7 @@
         Schedule schedule = new Schedule(numTransactions);
 
         for (int i = 0; i < numTransactions; i++) {
-            TransactionContext context = txnSimulator.beginTransaction();
+            ITransactionContext context = txnSimulator.beginTransaction();
             txnSimulator.executeTransactionOperation(context, schedule.getOperations()[i]);
             if (schedule.getWillCommit()[i]) {
                 txnSimulator.commitTransaction(context);
diff --git a/pom.xml b/pom.xml
index f4c46c6..c4e50c1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -50,7 +50,7 @@
             </property>
           </activation>
           <properties>
-            <test.heap.size>2047</test.heap.size>
+            <test.heap.size>2048</test.heap.size>
           </properties>
         </profile>
 
@@ -135,14 +135,6 @@
 			<url>http://obelix.ics.uci.edu/nexus/content/repositories/algebricks-snapshots/</url>
 		</repository>
 	</repositories>
-    <dependencies>
-        <dependency>
-            <groupId>net.sourceforge.cobertura</groupId>
-            <artifactId>cobertura</artifactId>
-            <version>1.9.4</version>
-            <optional>true</optional>
-        </dependency>
-    </dependencies>
         <dependencyManagement>
     	<dependencies>
     		<dependency>
@@ -232,6 +224,18 @@
 			<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
 			<version>${hyracks.version}</version>
 		</dependency>
+		<dependency>
+			<groupId>org.json</groupId>
+			<artifactId>json</artifactId>
+			<version>20090211</version>
+			<type>jar</type>
+		</dependency>
+		<dependency>
+			<groupId>javax.servlet</groupId>
+			<artifactId>servlet-api</artifactId>
+			<version>2.5</version>
+			<type>jar</type>
+		</dependency>
     	</dependencies>
     </dependencyManagement>
 </project>