Add DDL support for optionally associating meta record with dataset record.

-Fixed the TestExecutor.runScriptAndCompareWithResult() to compare fields
 after "Time"-prefixed fields.

Change-Id: I613316985dbd7f553d946f95c5b1a1df56914902
Reviewed-on: https://asterix-gerrit.ics.uci.edu/666
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
index 7272033..4c0818b 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
@@ -114,7 +114,7 @@
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
             throws AlgebricksException {
         AqlDataSource ds = new DatasetDataSource(datasetId, datasetId.getDataverseName(), datasetId.getDatasourceName(),
-                recordType, AqlDataSourceType.EXTERNAL_DATASET);
+                recordType, null /*external dataset doesn't have meta records.*/, AqlDataSourceType.EXTERNAL_DATASET);
         IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
         AbstractScanOperator as = (AbstractScanOperator) op;
         deliveredProperties = dspp.computePropertiesVector(as.getVariables());
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
index 1945be3..ab28e28 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
@@ -63,7 +63,8 @@
 public class UnnestToDataScanRule implements IAlgebraicRewriteRule {
 
     @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         return false;
     }
 
@@ -107,34 +108,40 @@
                 String datasetName = datasetReference.second;
                 Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
                 if (dataset == null) {
-                    throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse "
-                            + dataverseName);
+                    throw new AlgebricksException(
+                            "Could not find dataset " + datasetName + " in dataverse " + dataverseName);
                 }
 
                 AqlSourceId asid = new AqlSourceId(dataverseName, datasetName);
-
-                ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
+                List<LogicalVariable> variables = new ArrayList<LogicalVariable>();
 
                 if (dataset.getDatasetType() == DatasetType.INTERNAL) {
                     int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
                     for (int i = 0; i < numPrimaryKeys; i++) {
-                        v.add(context.newVar());
+                        variables.add(context.newVar());
                     }
                 }
-                v.add(unnest.getVariable());
+                variables.add(unnest.getVariable());
                 AqlDataSource dataSource = metadataProvider.findDataSource(asid);
-                DataSourceScanOperator scan = new DataSourceScanOperator(v, dataSource);
+                boolean hasMeta = dataSource.hasMeta();
+                if (hasMeta) {
+                    variables.add(context.newVar());
+                }
+                DataSourceScanOperator scan = new DataSourceScanOperator(variables, dataSource);
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
                 scanInpList.addAll(unnest.getInputs());
                 opRef.setValue(scan);
-                addPrimaryKey(v, context);
+                addPrimaryKey(variables, context);
                 context.computeAndSetTypeEnvironmentForOperator(scan);
 
                 // Adds equivalence classes --- one equivalent class between a primary key
                 // variable and a record field-access expression.
                 IAType[] schemaTypes = dataSource.getSchemaTypes();
-                ARecordType recordType = (ARecordType) schemaTypes[schemaTypes.length - 1];
-                EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(scan, v, recordType, dataset, context);
+                ARecordType recordType = (ARecordType) (hasMeta ? schemaTypes[schemaTypes.length - 2]
+                        : schemaTypes[schemaTypes.length - 1]);
+                ARecordType metaRecordType = (ARecordType) (hasMeta ? schemaTypes[schemaTypes.length - 1] : null);
+                EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(scan, variables, recordType,
+                        metaRecordType, dataset, context);
                 return true;
             }
 
@@ -166,8 +173,10 @@
                 v.add(unnest.getVariable());
 
                 String csLocations = metadataProvider.getConfig().get(FeedActivityDetails.COLLECT_LOCATIONS);
-                DataSourceScanOperator scan = new DataSourceScanOperator(v, createFeedDataSource(asid, targetDataset,
-                        sourceFeedName, subscriptionLocation, metadataProvider, policy, outputType, csLocations));
+                DataSourceScanOperator scan = new DataSourceScanOperator(v,
+                        createFeedDataSource(asid, targetDataset, sourceFeedName, subscriptionLocation,
+                                metadataProvider, policy, outputType,
+                                null /* TODO(Adbullah): to figure out the meta type name*/, csLocations));
 
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
                 scanInpList.addAll(unnest.getInputs());
@@ -194,16 +203,16 @@
 
     private AqlDataSource createFeedDataSource(AqlSourceId aqlId, String targetDataset, String sourceFeedName,
             String subscriptionLocation, AqlMetadataProvider metadataProvider, FeedPolicyEntity feedPolicy,
-            String outputType, String locations) throws AlgebricksException {
-        if (!aqlId.getDataverseName().equals(
-                metadataProvider.getDefaultDataverse() == null ? null : metadataProvider.getDefaultDataverse()
-                        .getDataverseName())) {
+            String outputType, String outputMetaType, String locations) throws AlgebricksException {
+        if (!aqlId.getDataverseName().equals(metadataProvider.getDefaultDataverse() == null ? null
+                : metadataProvider.getDefaultDataverse().getDataverseName())) {
             return null;
         }
         IAType feedOutputType = metadataProvider.findType(aqlId.getDataverseName(), outputType);
+        IAType feedOutputMetaType = metadataProvider.findType(aqlId.getDataverseName(), outputMetaType);
         Feed sourceFeed = metadataProvider.findFeed(aqlId.getDataverseName(), sourceFeedName);
 
-        FeedDataSource feedDataSource = new FeedDataSource(aqlId, targetDataset, feedOutputType,
+        FeedDataSource feedDataSource = new FeedDataSource(aqlId, targetDataset, feedOutputType, feedOutputMetaType,
                 AqlDataSource.AqlDataSourceType.FEED, sourceFeed.getFeedId(), sourceFeed.getFeedType(),
                 ConnectionLocation.valueOf(subscriptionLocation), locations.split(","));
         feedDataSource.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy);
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 3fdbc5c..1bbb13b 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -59,7 +59,6 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
@@ -124,8 +123,8 @@
             IOptimizationContext context) throws AlgebricksException {
         SelectOperator select = (SelectOperator) selectRef.getValue();
         Mutable<ILogicalExpression> conditionRef = select.getCondition();
-        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(conditionRef, subTree, null,
-                chosenIndex, analysisCtx, false, false, false, context);
+        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(conditionRef, subTree, null, chosenIndex,
+                analysisCtx, false, false, false, context);
         if (primaryIndexUnnestOp == null) {
             return false;
         }
@@ -185,8 +184,8 @@
             newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
         }
 
-        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(conditionRef, indexSubTree,
-                probeSubTree, chosenIndex, analysisCtx, true, isLeftOuterJoin, true, context);
+        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(conditionRef, indexSubTree, probeSubTree,
+                chosenIndex, analysisCtx, true, isLeftOuterJoin, true, context);
         if (primaryIndexUnnestOp == null) {
             return false;
         }
@@ -212,12 +211,13 @@
     }
 
     @Override
-    public ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalExpression> conditionRef, OptimizableOperatorSubTree indexSubTree,
-            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
-            boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context)
-                    throws AlgebricksException {
+    public ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalExpression> conditionRef,
+            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, boolean retainInput, boolean retainNull, boolean requiresBroadcast,
+            IOptimizationContext context) throws AlgebricksException {
         Dataset dataset = indexSubTree.dataset;
         ARecordType recordType = indexSubTree.recordType;
+        ARecordType metaRecordType = indexSubTree.metaRecordType;
         // we made sure indexSubTree has datasource scan
         AbstractDataSourceOperator dataSourceOp = (AbstractDataSourceOperator) indexSubTree.dataSourceRef.getValue();
         List<Pair<Integer, Integer>> exprAndVarList = analysisCtx.indexExprsAndVars.get(chosenIndex);
@@ -499,7 +499,7 @@
             // Adds equivalence classes --- one equivalent class between a primary key
             // variable and a record field-access expression.
             EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(primaryIndexUnnestOp,
-                    dataSourceOp.getVariables(), recordType, dataset, context);
+                    dataSourceOp.getVariables(), recordType, metaRecordType, dataset, context);
         } else {
             List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
             try {
@@ -527,7 +527,7 @@
             // Adds equivalence classes --- one equivalent class between a primary key
             // variable and a record field-access expression.
             EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(primaryIndexUnnestOp, scanVariables,
-                    recordType, dataset, context);
+                    recordType, metaRecordType, dataset, context);
         }
 
         return primaryIndexUnnestOp;
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 39a9ab8..be3dad8 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -68,6 +68,7 @@
     // Dataset and type metadata. Set in setDatasetAndTypeMetadata().
     public Dataset dataset = null;
     public ARecordType recordType = null;
+    public ARecordType metaRecordType = null;
 
     // Additional datasources can exist if IntroduceJoinAccessMethodRule has been applied.
     // (E.g. There are index-nested-loop-joins in the plan.)
@@ -264,10 +265,15 @@
             }
             rType = (ARecordType) itemType;
 
+            // Get the meta record type for that dataset.
+            IAType metaItemType = metadataProvider.findType(ds.getMetaItemTypeDataverseName(),
+                    ds.getMetaItemTypeName());
+
             // First index is always the primary datasource in this subtree.
             if (i == 0) {
                 dataset = ds;
                 recordType = rType;
+                metaRecordType = (ARecordType) metaItemType;
             } else {
                 ixJoinOuterAdditionalDatasets.add(ds);
                 ixJoinOuterAdditionalRecordTypes.add(rType);
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
index 9f98da0..5ede3bb 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
@@ -66,6 +66,8 @@
      *            is the record variable.
      * @param recordType
      *            , the record type of an index payload record.
+     * @param metaRecordType
+     *            , the type of a meta record associated with an index payload record.
      * @param dataset
      *            , the accessed dataset.
      * @param context
@@ -74,7 +76,7 @@
      */
     @SuppressWarnings("unchecked")
     public static void addEquivalenceClassesForPrimaryIndexAccess(ILogicalOperator operator,
-            List<LogicalVariable> indexSearchVars, ARecordType recordType, Dataset dataset,
+            List<LogicalVariable> indexSearchVars, ARecordType recordType, ARecordType metaRecordType, Dataset dataset,
             IOptimizationContext context) throws AlgebricksException {
         if (dataset.getDatasetDetails().getDatasetType() != DatasetType.INTERNAL) {
             return;
@@ -86,15 +88,30 @@
         for (int fieldIndex = 0; fieldIndex < fieldNames.length; ++fieldIndex) {
             fieldNameToIndexMap.put(fieldNames[fieldIndex], fieldIndex);
         }
-
-        LogicalVariable recordVar = indexSearchVars.get(indexSearchVars.size() - 1);
+        boolean hasMeta = dataset.hasMetaPart();
+        Map<String, Integer> metaFieldNameToIndexMap = new HashMap<>();
+        if (hasMeta) {
+            String[] metaFieldNames = metaRecordType.getFieldNames();
+            for (int metaFieldIndex = 0; metaFieldIndex < metaFieldNames.length; ++metaFieldIndex) {
+                metaFieldNameToIndexMap.put(metaFieldNames[metaFieldIndex], metaFieldIndex);
+            }
+        }
+        LogicalVariable recordVar = hasMeta ? indexSearchVars.get(indexSearchVars.size() - 2)
+                : indexSearchVars.get(indexSearchVars.size() - 1);
+        LogicalVariable metaRecordVar = hasMeta ? indexSearchVars.get(indexSearchVars.size() - 1) : null;
         for (int pkIndex = 0; pkIndex < primaryKey.size(); ++pkIndex) {
+            LogicalVariable referredRecordVar = recordVar;
             String pkFieldName = primaryKey.get(pkIndex).get(0);
-            int fieldIndexInRecord = fieldNameToIndexMap.get(pkFieldName);
+            Integer fieldIndexInRecord = fieldNameToIndexMap.get(pkFieldName);
+            if (fieldIndexInRecord == null && hasMeta) {
+                referredRecordVar = metaRecordVar;
+                pkFieldName = primaryKey.get(pkIndex).get(1);
+                fieldIndexInRecord = metaFieldNameToIndexMap.get(pkFieldName);
+            }
             LogicalVariable var = indexSearchVars.get(pkIndex);
             ILogicalExpression expr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(recordVar)),
+                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(referredRecordVar)),
                     new MutableObject<ILogicalExpression>(
                             new ConstantExpression(new AsterixConstantValue(new AInt32(fieldIndexInRecord)))));
             EquivalenceClass equivClass = new EquivalenceClass(SingletonList.newSingletonList(var), var,
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index dd4d6db..c635125 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -183,13 +183,15 @@
                     "Unable to load dataset " + clffs.getDatasetName() + " since it does not exist");
         }
         IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        IAType metaItemType = metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
+                dataset.getMetaItemTypeName());
         DatasetDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
                 stmt.getDatasetName());
         List<List<String>> partitionKeys = DatasetUtils.getPartitioningKeys(targetDatasource.getDataset());
 
         LoadableDataSource lds;
         try {
-            lds = new LoadableDataSource(dataset, itemType, clffs.getAdapter(), clffs.getProperties());
+            lds = new LoadableDataSource(dataset, itemType, metaItemType, clffs.getAdapter(), clffs.getProperties());
         } catch (IOException e) {
             throw new AlgebricksException(e);
         }
@@ -451,10 +453,11 @@
             throw new AlgebricksException("Cannot write output to an external dataset.");
         }
         AqlSourceId sourceId = new AqlSourceId(dataverseName, datasetName);
-        String itemTypeName = dataset.getItemTypeName();
-        IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), itemTypeName);
+        IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        IAType metaItemType = metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
+                dataset.getMetaItemTypeName());
         DatasetDataSource dataSource = new DatasetDataSource(sourceId, dataset.getDataverseName(),
-                dataset.getDatasetName(), itemType, AqlDataSourceType.INTERNAL_DATASET);
+                dataset.getDatasetName(), itemType, metaItemType, AqlDataSourceType.INTERNAL_DATASET);
 
         return dataSource;
     }
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
index 1c8db4f..f4691f1 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
@@ -35,6 +35,8 @@
  */
 public class ValidateUtil {
 
+    private static final String META = "meta()";
+
     /**
      * Validates the field that will be used as filter for the components of an LSM index.
      *
@@ -86,7 +88,7 @@
      * @throws AsterixException
      *             (if the validation failed), IOException
      */
-    public static List<IAType> validatePartitioningExpressions(ARecordType recType,
+    public static List<IAType> validatePartitioningExpressions(ARecordType recType, ARecordType metaRecType,
             List<List<String>> partitioningExprs, boolean autogenerated) throws AsterixException, IOException {
         List<IAType> partitioningExprTypes = new ArrayList<IAType>(partitioningExprs.size());
         if (autogenerated) {
@@ -105,7 +107,13 @@
         } else {
             for (int i = 0; i < partitioningExprs.size(); i++) {
                 List<String> fieldName = partitioningExprs.get(i);
-                IAType fieldType = recType.getSubFieldType(fieldName);
+                List<String> metaFieldName = null;
+                boolean useMeta = fieldName.get(0).equals(META);
+                if (useMeta) {
+                    metaFieldName = fieldName.subList(1, fieldName.size());
+                }
+                IAType fieldType = useMeta ? metaRecType.getSubFieldType(metaFieldName)
+                        : recType.getSubFieldType(fieldName);
                 switch (fieldType.getTypeTag()) {
                     case INT8:
                     case INT16:
@@ -156,15 +164,18 @@
             IAType fieldType = recType.getSubFieldType(fieldName);
             if (fieldType == null) {
                 fieldType = keyFieldTypes.get(pos);
-                if (keyFieldTypes.get(pos) == BuiltinType.ANULL)
+                if (keyFieldTypes.get(pos) == BuiltinType.ANULL) {
                     throw new AsterixException("A field with this name  \"" + fieldName + "\" could not be found.");
-            } else if (openFieldCompositeIdx)
+                }
+            } else if (openFieldCompositeIdx) {
                 throw new AsterixException("A closed field \"" + fieldName
                         + "\" could be only in a prefix part of the composite index, containing opened field.");
+            }
             if (keyFieldTypes.get(pos) != BuiltinType.ANULL
-                    && fieldType.getTypeTag() != keyFieldTypes.get(pos).getTypeTag())
+                    && fieldType.getTypeTag() != keyFieldTypes.get(pos).getTypeTag()) {
                 throw new AsterixException(
                         "A field \"" + fieldName + "\" is already defined with the type \"" + fieldType + "\"");
+            }
             switch (indexType) {
                 case BTREE:
                     switch (fieldType.getTypeTag()) {
diff --git a/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java b/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
index ea50221..87859dc 100644
--- a/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
+++ b/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
@@ -532,6 +532,8 @@
         DatasetType dsType = dd.getDatasetType();
         String itemTypeDataverseName = dd.getItemTypeDataverse().getValue();
         String itemTypeName = dd.getItemTypeName().getValue();
+        String metaItemTypeDataverseName = dd.getMetaItemTypeDataverse().getValue();
+        String metaItemTypeName = dd.getMetaItemTypeName().getValue();
         Identifier ngNameId = dd.getNodegroupName();
         String nodegroupName = getNodeGroupName(ngNameId, dd, dataverseName);
         String compactionPolicy = dd.getCompactionPolicy();
@@ -544,7 +546,8 @@
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
 
         MetadataLockManager.INSTANCE.createDatasetBegin(dataverseName, itemTypeDataverseName,
-                itemTypeDataverseName + "." + itemTypeName, nodegroupName, compactionPolicy,
+                itemTypeDataverseName + "." + itemTypeName, metaItemTypeDataverseName,
+                metaItemTypeDataverseName + "." + metaItemTypeName, nodegroupName, compactionPolicy,
                 dataverseName + "." + datasetName, defaultCompactionPolicy);
         Dataset dataset = null;
         try {
@@ -578,14 +581,24 @@
                 case INTERNAL: {
                     IAType itemType = dt.getDatatype();
                     if (itemType.getTypeTag() != ATypeTag.RECORD) {
-                        throw new AlgebricksException("Can only partition ARecord's.");
+                        throw new AlgebricksException("Dataset type has to be a record type.");
                     }
+
+                    IAType metaItemType = null;
+                    if (metaItemTypeDataverseName != null && metaItemTypeName != null) {
+                        metaItemType = metadataProvider.findType(metaItemTypeDataverseName, metaItemTypeName);
+                    }
+                    if (metaItemType != null && metaItemType.getTypeTag() != ATypeTag.RECORD) {
+                        throw new AlgebricksException("Dataset meta type has to be a record type.");
+                    }
+                    ARecordType metaRecType = (ARecordType) metaItemType;
+
                     List<List<String>> partitioningExprs = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
                             .getPartitioningExprs();
                     boolean autogenerated = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated();
                     ARecordType aRecordType = (ARecordType) itemType;
                     List<IAType> partitioningTypes = ValidateUtil.validatePartitioningExpressions(aRecordType,
-                            partitioningExprs, autogenerated);
+                            metaRecType, partitioningExprs, autogenerated);
 
                     List<String> filterField = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getFilterField();
                     if (filterField != null) {
@@ -622,9 +635,10 @@
             }
 
             //#. add a new dataset with PendingAddOp
-            dataset = new Dataset(dataverseName, datasetName, itemTypeDataverseName, itemTypeName, ngName,
-                    compactionPolicy, compactionPolicyProperties, datasetDetails, dd.getHints(), dsType,
-                    DatasetIdFactory.generateDatasetId(), IMetadataEntity.PENDING_ADD_OP);
+            dataset = new Dataset(dataverseName, datasetName, itemTypeDataverseName, itemTypeName,
+                    metaItemTypeDataverseName, metaItemTypeName, ngName, compactionPolicy, compactionPolicyProperties,
+                    datasetDetails, dd.getHints(), dsType, DatasetIdFactory.generateDatasetId(),
+                    IMetadataEntity.PENDING_ADD_OP);
             MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
 
             if (dd.getDatasetType() == DatasetType.INTERNAL) {
@@ -698,7 +712,8 @@
             throw e;
         } finally {
             MetadataLockManager.INSTANCE.createDatasetEnd(dataverseName, itemTypeDataverseName,
-                    itemTypeDataverseName + "." + itemTypeName, nodegroupName, compactionPolicy,
+                    itemTypeDataverseName + "." + itemTypeName, metaItemTypeDataverseName,
+                    metaItemTypeDataverseName + "." + metaItemTypeName, nodegroupName, compactionPolicy,
                     dataverseName + "." + datasetName, defaultCompactionPolicy);
         }
     }
@@ -1393,8 +1408,9 @@
                 MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
                 MetadataManager.INSTANCE.addDataset(mdTxnCtx,
                         new Dataset(dataverseName, datasetName, ds.getItemTypeDataverseName(), ds.getItemTypeName(),
-                                ds.getNodeGroupName(), ds.getCompactionPolicy(), ds.getCompactionPolicyProperties(),
-                                ds.getDatasetDetails(), ds.getHints(), ds.getDatasetType(), ds.getDatasetId(),
+                                ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName(), ds.getNodeGroupName(),
+                                ds.getCompactionPolicy(), ds.getCompactionPolicyProperties(), ds.getDatasetDetails(),
+                                ds.getHints(), ds.getDatasetType(), ds.getDatasetId(),
                                 IMetadataEntity.PENDING_DROP_OP));
 
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2935,7 +2951,9 @@
                     toDataset.getDatasetDetails().isTemp());
             DatasetDecl createToDataset = new DatasetDecl(new Identifier(dataverseNameTo),
                     pregelixStmt.getDatasetNameTo(), new Identifier(toDataset.getItemTypeDataverseName()),
-                    new Identifier(toDataset.getItemTypeName()), new Identifier(toDataset.getNodeGroupName()),
+                    new Identifier(toDataset.getItemTypeName()),
+                    new Identifier(toDataset.getMetaItemTypeDataverseName()),
+                    new Identifier(toDataset.getMetaItemTypeName()), new Identifier(toDataset.getNodeGroupName()),
                     toDataset.getCompactionPolicy(), toDataset.getCompactionPolicyProperties(), toDataset.getHints(),
                     toDataset.getDatasetType(), idd, false);
             this.handleCreateDatasetStatement(metadataProvider, createToDataset, hcc);
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.1.ddl.aql
new file mode 100644
index 0000000..6539dbc
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.1.ddl.aql
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type EmptyType as open {
+    id: string
+}
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(EmptyType)
+primary key id;
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.3.query.aql
new file mode 100644
index 0000000..079d075
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+for $x in dataset('Metadata.Dataset')
+where $x.DataverseName='test'
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.4.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.1.ddl.aql
new file mode 100644
index 0000000..cec01d2
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.1.ddl.aql
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse meta if exists;
+create dataverse meta;
+use dataverse meta;
+create type EmptyType as open {
+    id: string
+}
+
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(meta.EmptyType)
+primary key id;
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.3.query.aql
new file mode 100644
index 0000000..079d075
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+for $x in dataset('Metadata.Dataset')
+where $x.DataverseName='test'
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.4.ddl.aql
new file mode 100644
index 0000000..c7a256c
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.4.ddl.aql
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+drop dataverse meta if exists;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.1.ddl.aql
new file mode 100644
index 0000000..78bd7ab
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.1.ddl.aql
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type EmptyType as open {
+    id: string
+}
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(LineType)
+primary key id;
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.3.query.aql
new file mode 100644
index 0000000..079d075
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+for $x in dataset('Metadata.Dataset')
+where $x.DataverseName='test'
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.4.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.1.ddl.aql
new file mode 100644
index 0000000..869bc1e
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.1.ddl.aql
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(LineType)
+primary key meta().id;
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.3.query.aql
new file mode 100644
index 0000000..079d075
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+for $x in dataset('Metadata.Dataset')
+where $x.DataverseName='test'
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.4.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.1.ddl.aql
new file mode 100644
index 0000000..ddb42c8
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.1.ddl.aql
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type EmptyType as open {
+  "key":int32
+}
+
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(EmptyType)
+primary key meta()."key";
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.3.query.aql
new file mode 100644
index 0000000..079d075
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+for $x in dataset('Metadata.Dataset')
+where $x.DataverseName='test'
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.4.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-1/dataset_with_meta-1.1.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-1/dataset_with_meta-1.1.adm
new file mode 100644
index 0000000..776e07f
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-1/dataset_with_meta-1.1.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "Book", "DatatypeDataverseName": "test", "DatatypeName": "LineType", "DatasetType": "INTERNAL", "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "id" ] ], "PrimaryKey": [ [ "id" ] ], "Autogenerated": false }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Feb 24 17:32:57 PST 2016", "DatasetId": 101i32, "PendingOp": 0i32, "MetatypeDataverseName": "test", "MetatypeName": "EmptyType" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-2/dataset_with_meta-2.1.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-2/dataset_with_meta-2.1.adm
new file mode 100644
index 0000000..f3f00e9
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-2/dataset_with_meta-2.1.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "Book", "DatatypeDataverseName": "test", "DatatypeName": "LineType", "DatasetType": "INTERNAL", "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "id" ] ], "PrimaryKey": [ [ "id" ] ], "Autogenerated": false }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Feb 24 17:32:57 PST 2016", "DatasetId": 101i32, "PendingOp": 0i32, "MetatypeDataverseName": "meta", "MetatypeName": "EmptyType" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-3/dataset_with_meta-2.3.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-3/dataset_with_meta-2.3.adm
new file mode 100644
index 0000000..085aaa4
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-3/dataset_with_meta-2.3.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "Book", "DatatypeDataverseName": "test", "DatatypeName": "LineType", "DatasetType": "INTERNAL", "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "id" ] ], "PrimaryKey": [ [ "id" ] ], "Autogenerated": false }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Feb 24 17:32:57 PST 2016", "DatasetId": 101i32, "PendingOp": 0i32, "MetatypeDataverseName": "test", "MetatypeName": "LineType" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-4/dataset_with_meta-4.3.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-4/dataset_with_meta-4.3.adm
new file mode 100644
index 0000000..ec8c3bf
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-4/dataset_with_meta-4.3.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "Book", "DatatypeDataverseName": "test", "DatatypeName": "LineType", "DatasetType": "INTERNAL", "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "meta()", "id" ] ], "PrimaryKey": [ [ "meta()", "id" ] ], "Autogenerated": false }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Feb 24 22:29:39 PST 2016", "DatasetId": 101i32, "PendingOp": 0i32, "MetatypeDataverseName": "test", "MetatypeName": "LineType" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-5/dataset_with_meta-5.3.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-5/dataset_with_meta-5.3.adm
new file mode 100644
index 0000000..722f3a5
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-5/dataset_with_meta-5.3.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "Book", "DatatypeDataverseName": "test", "DatatypeName": "LineType", "DatasetType": "INTERNAL", "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "meta()", "key" ] ], "PrimaryKey": [ [ "meta()", "key" ] ], "Autogenerated": false }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Feb 24 22:55:29 PST 2016", "DatasetId": 105i32, "PendingOp": 0i32, "MetatypeDataverseName": "test", "MetatypeName": "EmptyType" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta01/meta01.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta01/meta01.1.adm
index 98f0152..a4c2eaa 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta01/meta01.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta01/meta01.1.adm
@@ -1,2 +1,2 @@
-{ "DataverseName": "Metadata", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Sat Nov 24 14:44:45 PST 2012" }
-{ "DataverseName": "testdv", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Sat Nov 24 14:45:14 PST 2012" }
+{ "DataverseName": "Metadata", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Wed Feb 24 18:38:04 PST 2016", "PendingOp": 0i32 }
+{ "DataverseName": "testdv", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Wed Feb 24 18:38:09 PST 2016", "PendingOp": 0i32 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm
index 82675b2..c92c3da 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm
@@ -1,2 +1,2 @@
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012" }
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012" }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Wed Feb 24 18:42:44 PST 2016", "PendingOp": 0i32 }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Wed Feb 24 18:42:44 PST 2016", "PendingOp": 0i32 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta12/meta12.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta12/meta12.1.adm
index aa1464a..b5f779e 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta12/meta12.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta12/meta12.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "dst01", "IndexName": "dst01", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Mon Sep 17 23:40:44 PDT 2012" }
+{ "DataverseName": "test", "DatasetName": "dst01", "IndexName": "dst01", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Wed Feb 24 18:42:45 PST 2016", "PendingOp": 0i32 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
index 117526a..9d7b9ce 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
@@ -1,2 +1,2 @@
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012", "SearchKeyType": [ "string" ] }
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012" }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32, "SearchKeyType": [ "string" ], "IsEnforced": true }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
index 9685606..bfd033c 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
@@ -1,2 +1,2 @@
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "RTREE", "SearchKey": [ [ "location" ] ], "IsPrimary": false, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012", "SearchKeyType": [ "point" ] }
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012" }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "RTREE", "SearchKey": [ [ "location" ] ], "IsPrimary": false, "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32, "SearchKeyType": [ "point" ], "IsEnforced": true }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataverse/metadata_dataverse.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataverse/metadata_dataverse.1.adm
index fc4acca..d5046f4 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataverse/metadata_dataverse.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataverse/metadata_dataverse.1.adm
@@ -1,3 +1,3 @@
-{ "DataverseName": "Metadata", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Thu Apr 04 21:10:48 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "test", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Thu Apr 04 21:10:55 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "testdv", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Thu Apr 04 21:10:52 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Wed Feb 24 18:43:46 PST 2016", "PendingOp": 0i32 }
+{ "DataverseName": "test", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32 }
+{ "DataverseName": "testdv", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32 }
diff --git a/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success.adm b/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success.adm
deleted file mode 100644
index 3d0da4d..0000000
--- a/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success.adm
+++ /dev/null
@@ -1,13 +0,0 @@
-{ "DataverseName": "custord", "DatatypeName": "AddressType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "street", "FieldType": "StreetType" }, { "FieldName": "city", "FieldType": "string" }, { "FieldName": "state", "FieldType": "string" }, { "FieldName": "zip", "FieldType": "int16" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "CustomerType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "cid", "FieldType": "int32" }, { "FieldName": "name", "FieldType": "string" }, { "FieldName": "age", "FieldType": "Field_age_in_CustomerType" }, { "FieldName": "address", "FieldType": "Field_address_in_CustomerType" }, { "FieldName": "interests", "FieldType": "Field_interests_in_CustomerType" }, { "FieldName": "children", "FieldType": "Field_children_in_CustomerType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "Field_address_in_CustomerType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "AddressType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "Field_age_in_CustomerType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "int32" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "Field_children_in_CustomerType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_children_in_CustomerType_ItemType" }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "Field_children_in_CustomerType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "name", "FieldType": "string" }, { "FieldName": "dob", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "Field_interests_in_CustomerType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "Field_items_in_OrderType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_items_in_OrderType_ItemType" }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "Field_items_in_OrderType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "number", "FieldType": "int64" }, { "FieldName": "storeIds", "FieldType": "Field_storeIds_in_Field_items_in_OrderType_ItemType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "Field_number_in_StreetType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "int32" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "Field_storeIds_in_Field_items_in_OrderType_ItemType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "int8", "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "OrderType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "oid", "FieldType": "int32" }, { "FieldName": "cid", "FieldType": "int32" }, { "FieldName": "orderstatus", "FieldType": "string" }, { "FieldName": "orderpriority", "FieldType": "string" }, { "FieldName": "clerk", "FieldType": "string" }, { "FieldName": "total", "FieldType": "float" }, { "FieldName": "items", "FieldType": "Field_items_in_OrderType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "StreetType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "number", "FieldType": "Field_number_in_StreetType" }, { "FieldName": "name", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
diff --git a/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success/verify_failure_previous_success.1.adm b/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success/verify_failure_previous_success.1.adm
index a9bf9ac..8bba3a4 100644
--- a/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success/verify_failure_previous_success.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success/verify_failure_previous_success.1.adm
@@ -7,4 +7,4 @@
 { "DataverseName": "custord", "DatatypeName": "OrderType_items", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "Record": null, "UnorderedList": null, "OrderedList": "OrderType_items_Item" }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
 { "DataverseName": "custord", "DatatypeName": "OrderType_items_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "number", "FieldType": "int64", "IsNullable": false }, { "FieldName": "storeIds", "FieldType": "OrderType_items_Item_storeIds", "IsNullable": false } ] }, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
 { "DataverseName": "custord", "DatatypeName": "OrderType_items_Item_storeIds", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "Record": null, "UnorderedList": "int8", "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "StreetType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "number", "FieldType": "int32", "IsNullable": true }, { "FieldName": "name", "FieldType": "string", "IsNullable": false } ] }, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
+{ "DataverseName": "custord", "DatatypeName": "StreetType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "number", "FieldType": "int32", "IsNullable": true }, { "FieldName": "name", "FieldType": "string", "IsNullable": false } ] }, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_subsequent_no_execution.adm b/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_subsequent_no_execution.adm
deleted file mode 100644
index f1d8d31..0000000
--- a/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_subsequent_no_execution.adm
+++ /dev/null
@@ -1 +0,0 @@
-{ "DataverseName": "custord", "DatasetName": "Customers", "IndexName": "Customers", "IndexStructure": "BTREE", "SearchKeyName": [ [ "cid" ], [ "name" ] ], "IsPrimary": true, "Timestamp": "Sat Nov 24 17:23:18 PST 2012", "SearchKeyType": [ ] }
diff --git a/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_subsequent_no_execution/verify_failure_subsequent_no_execution.1.adm b/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_subsequent_no_execution/verify_failure_subsequent_no_execution.1.adm
index c6a3d3b..68fb090 100644
--- a/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_subsequent_no_execution/verify_failure_subsequent_no_execution.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_subsequent_no_execution/verify_failure_subsequent_no_execution.1.adm
@@ -1 +1 @@
-{ "DataverseName": "custord", "DatasetName": "Customers", "IndexName": "Customers", "IndexStructure": "BTREE", "SearchKey": [ [ "cid" ], [ "name" ] ], "IsPrimary": true, "Timestamp": "Sat Nov 24 17:23:18 PST 2012" }
+{ "DataverseName": "custord", "DatasetName": "Customers", "IndexName": "Customers", "IndexStructure": "BTREE", "SearchKey": [ [ "cid" ], [ "name" ] ], "IsPrimary": true, "Timestamp": "Wed Feb 24 18:45:24 PST 2016", "PendingOp": 0i32 }
diff --git a/asterix-app/src/test/resources/metadata/testsuite.xml b/asterix-app/src/test/resources/metadata/testsuite.xml
index bba2ab0..550d93d 100644
--- a/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -19,6 +19,31 @@
 <test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
   <test-group name="basic">
     <test-case FilePath="basic">
+      <compilation-unit name="dataset_with_meta-1">
+        <output-dir compare="Text">dataset_with_meta-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="dataset_with_meta-2">
+        <output-dir compare="Text">dataset_with_meta-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="dataset_with_meta-3">
+        <output-dir compare="Text">dataset_with_meta-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="dataset_with_meta-4">
+        <output-dir compare="Text">dataset_with_meta-4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="dataset_with_meta-5">
+        <output-dir compare="Text">dataset_with_meta-5</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
       <compilation-unit name="meta01">
         <output-dir compare="Text">meta01</output-dir>
       </compilation-unit>
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.1.ddl.aql
new file mode 100644
index 0000000..ca5a534
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.1.ddl.aql
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type EmptyType as open {
+  "key":int32
+}
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(EmptyType)
+primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.3.query.aql
new file mode 100644
index 0000000..43150da
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.3.query.aql
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+for $x in dataset Book
+return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.4.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.4.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.1.ddl.aql
new file mode 100644
index 0000000..a341b29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.1.ddl.aql
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type EmptyType as open {
+  "key":int32
+}
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(EmptyType)
+primary key meta()."key";
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.3.query.aql
new file mode 100644
index 0000000..43150da
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.3.query.aql
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse test;
+
+for $x in dataset Book
+return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.4.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.4.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterix-app/src/test/resources/runtimets/results/feeds/feed-with-external-parser/feed-with-external-parser.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feed-with-external-parser/feed-with-external-parser.1.adm
index a155b33..c0709a00 100644
--- a/asterix-app/src/test/resources/runtimets/results/feeds/feed-with-external-parser/feed-with-external-parser.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feed-with-external-parser/feed-with-external-parser.1.adm
@@ -1,99 +1,99 @@
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#48968872.0#1445354636", "StatsLifetimeStarter": 572059, "JobStartDate": 1445362267, "SubmitEventNotes": "DAG Node: fabp4-0002+fabp4-0002", "JobStatus": 4, "LeaveJobInQueue": false, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.104.119.175", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1445561276, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "ScheddBday": 1445383086, "RemoteWallClockTime": 769511.0d, "WantCheckpoint": false, "In": "/dev/null", "LastVacateTime": 1445546251, "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 32543, "EnteredCurrentStatus": 1446133322, "ResidentSetSize_RAW": 100432, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/ssericksen/dude-14-xdock/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 571737.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 690056, "BytesSent": 3113566.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133322, "ProcId": 0, "ImageSize": 750000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 12, "RecentBlockReads": 0, "SpooledOutputFiles": "ChtcWrapperfabp4-0002.out,AuditLog.fabp4-0002,poses.mol2,CURLTIME_4057178,harvest.log,time_elapsed.log,surf_scores.txt,CURLTIME_38803,count.log,fabp4-0002.out,CURLTIME_253463", "NumJobReconnects": 1, "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT_OR_EVICT", "JobCurrentStartExecutingDate": 1445561278, "ExitBySignal": false, "LastMatchTime": 1445561276, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 6, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 48940805, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 6, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 572046, "ExecutableSize_RAW": 6, "LastRejMatchReason": "no match found", "LastSuspensionTime": 0, "UserLog": "/home/ssericksen/dude-14-xdock/ChtcRun/dude14-surf-out-esr1/fabp4-0002/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 769511.0d, "LastJobLeaseRenewal": 1446133322, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 8.7351688E7d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "fabp4-0002+fabp4-0002", "PeriodicRelease": "error", "JobRunCount": 7, "LastRemotePool": "condor.biochem.wisc.edu:9618?sock=collector", "JobLastStartDate": 1445546257, "LastRemoteHost": "slot1@cluster-0008.biochem.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 0.0d, "TransferInput": "/home/ssericksen/dude-14-xdock/ChtcRun/dude14-surf-in/fabp4-0002/,/home/ssericksen/dude-14-xdock/ChtcRun/dude14-surf-in/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133322, "StreamErr": false, "is_resumable": true, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 7, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/ssericksen/dude-14-xdock/ChtcRun/dude14-surf-out-esr1/./mydag.dag.nodes.log", "Owner": "ssericksen", "Requirements": "undefined", "DiskUsage": 35000, "LastRejMatchTime": 1445375317, "JobLeaseDuration": 2400, "ClusterId": 48968872, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 572046.0d, "Args": "--type=Other --cmdtorun=surflex_run_DUDE_v1.8_esr1.sh --unique=fabp4-0002 --", "Environment": "", "LastPublicClaimId": "<128.104.119.175:9618>#1444067179#3317#...", "Iwd": "/home/ssericksen/dude-14-xdock/ChtcRun/dude14-surf-out-esr1/fabp4-0002", "QDate": 1445354636, "CurrentHosts": 0, "User": "ssericksen@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49573720.0#1445938922", "StatsLifetimeStarter": 190245, "JobStartDate": 1445943852, "SubmitEventNotes": "DAG Node: 180+180", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.72", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1445943852, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 190247.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134099, "ResidentSetSize_RAW": 123680, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 185236.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30766.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134099, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3753852,ChtcWrapper180.out,AuditLog.180,simu_3_180.txt,harvest.log,180.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1445943853, "ExitBySignal": false, "LastMatchTime": 1445943852, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49572657, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 190247, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally/Simulation_condor/model_3/180/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 190247.0d, "LastJobLeaseRenewal": 1446134099, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284367.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "180+180", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e272.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 1835.0d, "TransferInput": "/home/xguo23/finally/Simulation_condor/data/180/,/home/xguo23/finally/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134099, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49573720, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 190247.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=180 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.72:29075>#1444753997#6000#...", "Iwd": "/home/xguo23/finally/Simulation_condor/model_3/180", "QDate": 1445938922, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49581952.0#1446105329", "StatsLifetimeStarter": 27674, "JobStartDate": 1446106061, "SubmitEventNotes": "DAG Node: 40+40", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.86", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446106061, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 27676.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133737, "ResidentSetSize_RAW": 127252, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 27510.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30584.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133737, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_369560,ChtcWrapper40.out,AuditLog.40,simu_3_40.txt,harvest.log,40.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446106063, "ExitBySignal": false, "LastMatchTime": 1446106061, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 27676, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/40/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 27676.0d, "LastJobLeaseRenewal": 1446133737, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285053.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "40+40", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e286.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 105.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/40/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133737, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49581952, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 27676.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=40 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.86:32129>#1444759888#6329#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/40", "QDate": 1446105329, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49581985.0#1446105368", "StatsLifetimeStarter": 26354, "JobStartDate": 1446106289, "SubmitEventNotes": "DAG Node: 36+36", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.244.249", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446106289, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 26357.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132646, "ResidentSetSize_RAW": 127452, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 26239.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31898.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132646, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1151700,ChtcWrapper36.out,AuditLog.36,simu_3_36.txt,harvest.log,36.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446106291, "ExitBySignal": false, "LastMatchTime": 1446106289, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 26357, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/36/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 26357.0d, "LastJobLeaseRenewal": 1446132646, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285053.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "36+36", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e457.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 96.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/36/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132646, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49581985, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 26357.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=36 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.244.249:28476>#1444685646#10655#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/36", "QDate": 1446105368, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49581989.0#1446105374", "StatsLifetimeStarter": 27490, "JobStartDate": 1446106290, "SubmitEventNotes": "DAG Node: 82+82", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.233", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446106290, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 27491.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133781, "ResidentSetSize_RAW": 126932, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 27288.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30553.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133782, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_4096502,ChtcWrapper82.out,AuditLog.82,simu_3_82.txt,harvest.log,82.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446106291, "ExitBySignal": false, "LastMatchTime": 1446106290, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 27491, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/82/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 27491.0d, "LastJobLeaseRenewal": 1446133781, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285053.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "82+82", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e433.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 173.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/82/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133781, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49581989, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 27491.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=82 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.233:28601>#1443991451#13496#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/82", "QDate": 1446105374, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582049.0#1446105441", "StatsLifetimeStarter": 26296, "JobStartDate": 1446106482, "SubmitEventNotes": "DAG Node: 112+112", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.245", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446106482, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 26298.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132780, "ResidentSetSize_RAW": 126892, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 26097.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31904.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132780, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2601607,ChtcWrapper112.out,AuditLog.112,simu_3_112.txt,harvest.log,112.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446106484, "ExitBySignal": false, "LastMatchTime": 1446106482, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 26298, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/112/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 26298.0d, "LastJobLeaseRenewal": 1446132780, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "112+112", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e445.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 164.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/112/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132780, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582049, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 26298.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=112 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.245:48407>#1443991450#14631#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/112", "QDate": 1446105441, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582050.0#1446105441", "StatsLifetimeStarter": 27141, "JobStartDate": 1446106482, "SubmitEventNotes": "DAG Node: 301+301", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.172", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446106482, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 27143.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133625, "ResidentSetSize_RAW": 126464, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 26895.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31905.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133625, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2158419,ChtcWrapper301.out,AuditLog.301,simu_3_301.txt,harvest.log,301.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446106484, "ExitBySignal": false, "LastMatchTime": 1446106482, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 27143, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/301/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 27143.0d, "LastJobLeaseRenewal": 1446133625, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "301+301", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e372.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 201.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/301/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133625, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582050, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 27143.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=301 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.172:19856>#1444760019#9307#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/301", "QDate": 1446105441, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582065.0#1446105458", "StatsLifetimeStarter": 25606, "JobStartDate": 1446107042, "SubmitEventNotes": "DAG Node: 401+401", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.206", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107042, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 25607.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132649, "ResidentSetSize_RAW": 126608, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 25478.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30661.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132649, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1160521,ChtcWrapper401.out,AuditLog.401,simu_3_401.txt,harvest.log,401.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107043, "ExitBySignal": false, "LastMatchTime": 1446107042, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25607, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/401/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25607.0d, "LastJobLeaseRenewal": 1446132649, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "401+401", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e406.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 89.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/401/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132649, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582065, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25607.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=401 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.206:27946>#1443991437#15826#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/401", "QDate": 1446105458, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582094.0#1446105491", "StatsLifetimeStarter": 25168, "JobStartDate": 1446107489, "SubmitEventNotes": "DAG Node: 106+106", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.83", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107489, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 75000, "RemoteWallClockTime": 25169.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 4, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132658, "ResidentSetSize_RAW": 72016, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24949.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 119520, "BytesSent": 30486.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446132658, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 86, "SpooledOutputFiles": "CURLTIME_122139,ChtcWrapper106.out,AuditLog.106,simu_3_106.txt,harvest.log,106.out", "BlockWriteKbytes": 4, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107490, "ExitBySignal": false, "LastMatchTime": 1446107489, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 665, "DAGManJobId": 49581933, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 26620, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25169, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/106/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25169.0d, "LastJobLeaseRenewal": 1446132658, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "106+106", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c064.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 204.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/106/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132658, "StreamErr": false, "RecentBlockReadKbytes": 960, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582094, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25169.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=106 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.83:25899>#1445308581#1240#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/106", "QDate": 1446105491, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582098.0#1446105492", "StatsLifetimeStarter": 26020, "JobStartDate": 1446107489, "SubmitEventNotes": "DAG Node: 304+304", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.223", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107489, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 26022.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133511, "ResidentSetSize_RAW": 128776, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 25844.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31801.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133511, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3651606,ChtcWrapper304.out,AuditLog.304,simu_3_304.txt,harvest.log,304.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107491, "ExitBySignal": false, "LastMatchTime": 1446107489, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 26022, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/304/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 26022.0d, "LastJobLeaseRenewal": 1446133511, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "304+304", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e423.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 143.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/304/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133511, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582098, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 26022.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=304 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.223:13467>#1444760039#6376#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/304", "QDate": 1446105492, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582113.0#1446105509", "StatsLifetimeStarter": 26044, "JobStartDate": 1446107490, "SubmitEventNotes": "DAG Node: 206+206", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.120", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107490, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 26045.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133535, "ResidentSetSize_RAW": 126460, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 25939.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30596.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133535, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_522843,ChtcWrapper206.out,AuditLog.206,simu_3_206.txt,harvest.log,206.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107491, "ExitBySignal": false, "LastMatchTime": 1446107490, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 26045, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/206/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 26045.0d, "LastJobLeaseRenewal": 1446133535, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "206+206", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e320.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 87.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/206/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133535, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582113, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 26045.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=206 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.120:45185>#1443991409#14238#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/206", "QDate": 1446105509, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582119.0#1446105519", "StatsLifetimeStarter": 24928, "JobStartDate": 1446107490, "SubmitEventNotes": "DAG Node: 152+152", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.242", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107490, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24930.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132420, "ResidentSetSize_RAW": 128972, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24742.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30431.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132420, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_674,ChtcWrapper152.out,AuditLog.152,simu_3_152.txt,harvest.log,152.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107491, "ExitBySignal": false, "LastMatchTime": 1446107490, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24930, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/152/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24930.0d, "LastJobLeaseRenewal": 1446132420, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "152+152", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e442.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 156.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/152/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132420, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582119, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24930.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=152 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.242:38884>#1443991450#10374#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/152", "QDate": 1446105519, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582148.0#1446105547", "StatsLifetimeStarter": 26230, "JobStartDate": 1446107686, "SubmitEventNotes": "DAG Node: 162+162", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.170", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107686, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 26233.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133919, "ResidentSetSize_RAW": 126384, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 26088.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30612.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133919, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1114551,ChtcWrapper162.out,AuditLog.162,simu_3_162.txt,harvest.log,162.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107688, "ExitBySignal": false, "LastMatchTime": 1446107686, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 26233, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/162/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 26233.0d, "LastJobLeaseRenewal": 1446133919, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "162+162", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e370.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 96.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/162/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133919, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582148, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 26233.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=162 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.170:9482>#1443991414#13008#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/162", "QDate": 1446105547, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582154.0#1446105553", "StatsLifetimeStarter": 25874, "JobStartDate": 1446107686, "SubmitEventNotes": "DAG Node: 333+333", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.120", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107686, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 25876.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133562, "ResidentSetSize_RAW": 125740, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 25692.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30542.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133562, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_523030,ChtcWrapper333.out,AuditLog.333,simu_3_333.txt,harvest.log,333.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107688, "ExitBySignal": false, "LastMatchTime": 1446107686, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25876, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/333/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25876.0d, "LastJobLeaseRenewal": 1446133562, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "333+333", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e320.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 157.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/333/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133562, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582154, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25876.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=333 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.120:45185>#1443991409#14242#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/333", "QDate": 1446105553, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582177.0#1446105581", "StatsLifetimeStarter": 25025, "JobStartDate": 1446108665, "SubmitEventNotes": "DAG Node: 145+145", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.57", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108665, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 75000, "RemoteWallClockTime": 25026.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 4, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133691, "ResidentSetSize_RAW": 73308, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24770.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 120972, "BytesSent": 28290.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446133691, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 146, "SpooledOutputFiles": "CURLTIME_4179033,ChtcWrapper145.out,AuditLog.145,simu_3_145.txt,harvest.log,145.out", "BlockWriteKbytes": 4, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108666, "ExitBySignal": false, "LastMatchTime": 1446108665, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 796, "DAGManJobId": 49581933, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 28476, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25026, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/145/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25026.0d, "LastJobLeaseRenewal": 1446133691, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "145+145", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c038.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 217.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/145/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133691, "StreamErr": false, "RecentBlockReadKbytes": 1932, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582177, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25026.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=145 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.57:49793>#1445322694#1541#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/145", "QDate": 1446105581, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582178.0#1446105581", "StatsLifetimeStarter": 24871, "JobStartDate": 1446108666, "SubmitEventNotes": "DAG Node: 154+154", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.158", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108666, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24874.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133540, "ResidentSetSize_RAW": 125792, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24626.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30559.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133540, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1382128,ChtcWrapper154.out,AuditLog.154,simu_3_154.txt,harvest.log,154.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108668, "ExitBySignal": false, "LastMatchTime": 1446108666, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24874, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/154/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24874.0d, "LastJobLeaseRenewal": 1446133540, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "154+154", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e358.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 183.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/154/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133540, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582178, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24874.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=154 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.158:24962>#1444759998#9379#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/154", "QDate": 1446105581, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582181.0#1446105586", "StatsLifetimeStarter": 25146, "JobStartDate": 1446108665, "SubmitEventNotes": "DAG Node: 181+181", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.102", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108665, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 25148.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133813, "ResidentSetSize_RAW": 125368, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24957.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30557.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133813, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3925831,ChtcWrapper181.out,AuditLog.181,simu_3_181.txt,harvest.log,181.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108667, "ExitBySignal": false, "LastMatchTime": 1446108665, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25148, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/181/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25148.0d, "LastJobLeaseRenewal": 1446133813, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "181+181", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e302.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 148.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/181/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133813, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582181, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25148.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=181 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.102:26944>#1443991374#13401#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/181", "QDate": 1446105586, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582187.0#1446105592", "StatsLifetimeStarter": 25238, "JobStartDate": 1446108666, "SubmitEventNotes": "DAG Node: 343+343", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.141", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108666, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 25241.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133907, "ResidentSetSize_RAW": 127540, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 25080.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31798.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133907, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2577757,ChtcWrapper343.out,AuditLog.343,simu_3_343.txt,harvest.log,343.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108668, "ExitBySignal": false, "LastMatchTime": 1446108666, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25241, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/343/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25241.0d, "LastJobLeaseRenewal": 1446133907, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "343+343", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e341.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 127.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/343/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133907, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582187, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25241.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=343 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.141:7534>#1444673425#9467#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/343", "QDate": 1446105592, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582211.0#1446105612", "StatsLifetimeStarter": 23957, "JobStartDate": 1446108667, "SubmitEventNotes": "DAG Node: 5+5", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.105", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108667, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 23958.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132625, "ResidentSetSize_RAW": 125804, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 23804.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30545.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132625, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1839815,ChtcWrapper5.out,AuditLog.5,simu_3_5.txt,harvest.log,5.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108668, "ExitBySignal": false, "LastMatchTime": 1446108667, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 23958, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/5/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 23958.0d, "LastJobLeaseRenewal": 1446132625, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284627.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "5+5", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e305.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 120.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/5/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132625, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582211, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 23958.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=5 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.105:48578>#1445357425#5008#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/5", "QDate": 1446105612, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582222.0#1446105617", "StatsLifetimeStarter": 24898, "JobStartDate": 1446108828, "SubmitEventNotes": "DAG Node: 6+6", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.67", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108828, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24900.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133728, "ResidentSetSize_RAW": 126592, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24729.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30543.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133728, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1151628,ChtcWrapper6.out,AuditLog.6,simu_3_6.txt,harvest.log,6.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108830, "ExitBySignal": false, "LastMatchTime": 1446108828, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24900, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/6/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24900.0d, "LastJobLeaseRenewal": 1446133728, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284627.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "6+6", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e267.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 129.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/6/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133728, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582222, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24900.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=6 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.67:65111>#1444759823#5994#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/6", "QDate": 1446105617, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582230.0#1446105620", "StatsLifetimeStarter": 24122, "JobStartDate": 1446108827, "SubmitEventNotes": "DAG Node: 182+182", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.62", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108827, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24124.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132951, "ResidentSetSize_RAW": 125656, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 23953.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30489.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132951, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_511744,ChtcWrapper182.out,AuditLog.182,simu_3_182.txt,harvest.log,182.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108829, "ExitBySignal": false, "LastMatchTime": 1446108827, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24124, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/182/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24124.0d, "LastJobLeaseRenewal": 1446132951, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "182+182", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e262.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 126.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/182/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132951, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582230, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24124.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=182 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.62:1966>#1444680938#10248#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/182", "QDate": 1446105620, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582236.0#1446105622", "StatsLifetimeStarter": 25080, "JobStartDate": 1446108827, "SubmitEventNotes": "DAG Node: 10+10", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.245", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108827, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 25082.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133909, "ResidentSetSize_RAW": 126500, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24966.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30659.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133909, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2607549,ChtcWrapper10.out,AuditLog.10,simu_3_10.txt,harvest.log,10.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108829, "ExitBySignal": false, "LastMatchTime": 1446108827, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25082, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/10/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25082.0d, "LastJobLeaseRenewal": 1446133909, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "10+10", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e445.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 83.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/10/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133909, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582236, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25082.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=10 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.245:48407>#1443991450#14655#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/10", "QDate": 1446105622, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582256.0#1446105629", "StatsLifetimeStarter": 24797, "JobStartDate": 1446108828, "SubmitEventNotes": "DAG Node: 23+23", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.244.247", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108828, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 24798.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133626, "ResidentSetSize_RAW": 126760, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24631.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30534.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133627, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_908930,ChtcWrapper23.out,AuditLog.23,simu_3_23.txt,harvest.log,23.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108829, "ExitBySignal": false, "LastMatchTime": 1446108828, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24798, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/23/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24798.0d, "LastJobLeaseRenewal": 1446133626, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "23+23", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e455.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 132.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/23/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133626, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582256, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24798.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=23 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.244.247:44193>#1444685638#5758#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/23", "QDate": 1446105629, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582261.0#1446105631", "StatsLifetimeStarter": 25132, "JobStartDate": 1446108995, "SubmitEventNotes": "DAG Node: 407+407", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38210, "StartdPrincipal": "execute-side@matchsession/128.104.55.48", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108995, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 100000, "RemoteWallClockTime": 25133.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134128, "ResidentSetSize_RAW": 76112, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24776.0d, "BlockWrites": 4, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 123648, "BytesSent": 30561.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134128, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 313, "SpooledOutputFiles": "harvest.log,ChtcWrapper407.out,AuditLog.407,CURLTIME_1861323,407.out,simu_3_407.txt", "BlockWriteKbytes": 16, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108996, "ExitBySignal": false, "LastMatchTime": 1446108995, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 906, "DAGManJobId": 49581933, "MemoryUsage": 97, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 30280, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25133, "ExecutableSize_RAW": 6, "LastRejMatchReason": "PREEMPTION_REQUIREMENTS == False ", "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/407/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25133.0d, "LastJobLeaseRenewal": 1446134128, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "407+407", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c029.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 277.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/407/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134128, "StreamErr": false, "RecentBlockReadKbytes": 3976, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "LastRejMatchTime": 1446108994, "JobLeaseDuration": 2400, "ClusterId": 49582261, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25133.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=407 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.48:26476>#1445344800#1604#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/407", "QDate": 1446105631, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582281.0#1446105639", "StatsLifetimeStarter": 23559, "JobStartDate": 1446109353, "SubmitEventNotes": "DAG Node: 24+24", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.149", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109353, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 23560.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132913, "ResidentSetSize_RAW": 127800, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 23403.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27904.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132913, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2096182,ChtcWrapper24.out,AuditLog.24,simu_3_24.txt,harvest.log,24.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446109354, "ExitBySignal": false, "LastMatchTime": 1446109353, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 23560, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/24/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 23560.0d, "LastJobLeaseRenewal": 1446132913, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "24+24", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e349.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 118.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/24/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132913, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582281, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 23560.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=24 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.149:6629>#1443991419#14390#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/24", "QDate": 1446105639, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582283.0#1446105640", "StatsLifetimeStarter": 24295, "JobStartDate": 1446109353, "SubmitEventNotes": "DAG Node: 33+33", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.75", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109353, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24297.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133650, "ResidentSetSize_RAW": 126564, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24147.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31684.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133650, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3304508,ChtcWrapper33.out,AuditLog.33,simu_3_33.txt,harvest.log,33.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446109354, "ExitBySignal": false, "LastMatchTime": 1446109353, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24297, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/33/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24297.0d, "LastJobLeaseRenewal": 1446133650, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "33+33", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e275.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 135.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/33/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133650, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582283, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24297.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=33 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.75:36755>#1444759846#8529#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/33", "QDate": 1446105640, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582308.0#1446105649", "StatsLifetimeStarter": 23044, "JobStartDate": 1446109803, "SubmitEventNotes": "DAG Node: 25+25", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.190", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109803, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 23045.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132848, "ResidentSetSize_RAW": 126180, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22891.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30497.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132848, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_4129250,ChtcWrapper25.out,AuditLog.25,simu_3_25.txt,harvest.log,25.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446109804, "ExitBySignal": false, "LastMatchTime": 1446109803, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 23045, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/25/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 23045.0d, "LastJobLeaseRenewal": 1446132848, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "25+25", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e390.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 132.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/25/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132848, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582308, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 23045.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=25 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.190:40807>#1443991430#14737#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/25", "QDate": 1446105649, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582313.0#1446105651", "StatsLifetimeStarter": 24043, "JobStartDate": 1446109803, "SubmitEventNotes": "DAG Node: 61+61", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.92", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109803, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24044.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133847, "ResidentSetSize_RAW": 128692, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 23894.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30500.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133847, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3652530,ChtcWrapper61.out,AuditLog.61,simu_3_61.txt,harvest.log,61.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446109804, "ExitBySignal": false, "LastMatchTime": 1446109803, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24044, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/61/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24044.0d, "LastJobLeaseRenewal": 1446133847, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "61+61", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e292.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 113.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/61/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133847, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582313, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24044.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=61 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.92:44347>#1444759907#8412#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/61", "QDate": 1446105651, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582323.0#1446105655", "StatsLifetimeStarter": 22981, "JobStartDate": 1446109998, "SubmitEventNotes": "DAG Node: 61+61", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.83", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109998, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 75000, "RemoteWallClockTime": 22983.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 4, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132981, "ResidentSetSize_RAW": 72244, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22740.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 119748, "BytesSent": 30533.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446132981, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 71, "SpooledOutputFiles": "CURLTIME_127008,ChtcWrapper61.out,AuditLog.61,simu_3_61.txt,harvest.log,61.out", "BlockWriteKbytes": 4, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110000, "ExitBySignal": false, "LastMatchTime": 1446109998, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 808, "DAGManJobId": 49582206, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 37312, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22983, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/61/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22983.0d, "LastJobLeaseRenewal": 1446132981, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "61+61", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c064.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 197.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/61/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132981, "StreamErr": false, "RecentBlockReadKbytes": 848, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582323, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22983.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=61 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.83:25899>#1445308581#1248#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/61", "QDate": 1446105655, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582339.0#1446105660", "StatsLifetimeStarter": 22784, "JobStartDate": 1446109999, "SubmitEventNotes": "DAG Node: 35+35", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.177", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109999, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22787.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132786, "ResidentSetSize_RAW": 125340, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22613.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30552.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132786, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3107604,ChtcWrapper35.out,AuditLog.35,simu_3_35.txt,harvest.log,35.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110001, "ExitBySignal": false, "LastMatchTime": 1446109999, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22787, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/35/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22787.0d, "LastJobLeaseRenewal": 1446132786, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "35+35", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e377.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 130.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/35/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132786, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582339, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22787.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=35 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.177:46087>#1443991411#13647#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/35", "QDate": 1446105660, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582357.0#1446105667", "StatsLifetimeStarter": 22635, "JobStartDate": 1446110000, "SubmitEventNotes": "DAG Node: 18+18", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.248", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110000, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22636.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132636, "ResidentSetSize_RAW": 127300, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22506.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27904.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132636, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2816308,ChtcWrapper18.out,AuditLog.18,simu_3_18.txt,harvest.log,18.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110001, "ExitBySignal": false, "LastMatchTime": 1446110000, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22636, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/18/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22636.0d, "LastJobLeaseRenewal": 1446132636, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "18+18", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e448.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 111.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/18/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132636, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582357, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22636.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=18 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.248:41700>#1443991446#11545#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/18", "QDate": 1446105667, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582358.0#1446105667", "StatsLifetimeStarter": 22588, "JobStartDate": 1446110000, "SubmitEventNotes": "DAG Node: 36+36", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.226", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110000, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22590.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132590, "ResidentSetSize_RAW": 125920, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22431.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30587.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132590, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_533867,ChtcWrapper36.out,AuditLog.36,simu_3_36.txt,harvest.log,36.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110002, "ExitBySignal": false, "LastMatchTime": 1446110000, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22590, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/36/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22590.0d, "LastJobLeaseRenewal": 1446132590, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "36+36", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e426.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 130.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/36/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132590, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582358, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22590.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=36 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.226:11484>#1443991456#11499#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/36", "QDate": 1446105667, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582391.0#1446105679", "StatsLifetimeStarter": 23041, "JobStartDate": 1446110203, "SubmitEventNotes": "DAG Node: 83+83", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.244.249", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110203, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 23043.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133246, "ResidentSetSize_RAW": 127748, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22845.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27908.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133246, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1162426,ChtcWrapper83.out,AuditLog.83,simu_3_83.txt,harvest.log,83.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110205, "ExitBySignal": false, "LastMatchTime": 1446110203, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 23043, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/83/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 23043.0d, "LastJobLeaseRenewal": 1446133246, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "83+83", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e457.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 142.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/83/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133246, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582391, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 23043.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=83 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.244.249:28476>#1444685646#10673#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/83", "QDate": 1446105679, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582404.0#1446105684", "StatsLifetimeStarter": 22108, "JobStartDate": 1446110586, "SubmitEventNotes": "DAG Node: 29+29", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.21", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110586, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 75000, "RemoteWallClockTime": 22109.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132695, "ResidentSetSize_RAW": 70692, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21939.0d, "BlockWrites": 2, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 118196, "BytesSent": 28013.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132695, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 3, "SpooledOutputFiles": "harvest.log,simu_3_29.txt,ChtcWrapper29.out,AuditLog.29,29.out,CURLTIME_3320245", "BlockWriteKbytes": 8, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110587, "ExitBySignal": false, "LastMatchTime": 1446110586, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 811, "DAGManJobId": 49582200, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 39868, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22109, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/29/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22109.0d, "LastJobLeaseRenewal": 1446132695, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "29+29", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c002.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 129.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/29/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132695, "StreamErr": false, "RecentBlockReadKbytes": 12, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582404, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22109.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=29 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.21:40483>#1445289732#1640#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/29", "QDate": 1446105684, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582416.0#1446105688", "StatsLifetimeStarter": 21892, "JobStartDate": 1446110790, "SubmitEventNotes": "DAG Node: 47+47", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.94", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110790, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 21894.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132684, "ResidentSetSize_RAW": 126208, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21739.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 29074.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132684, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_143217,ChtcWrapper47.out,AuditLog.47,simu_3_47.txt,harvest.log,47.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110792, "ExitBySignal": false, "LastMatchTime": 1446110790, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21894, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/47/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21894.0d, "LastJobLeaseRenewal": 1446132684, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "47+47", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e294.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 121.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/47/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132684, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582416, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21894.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=47 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.94:64588>#1444759915#9064#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/47", "QDate": 1446105688, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582421.0#1446105690", "StatsLifetimeStarter": 22909, "JobStartDate": 1446110790, "SubmitEventNotes": "DAG Node: 66+66", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.244.247", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110790, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22911.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133701, "ResidentSetSize_RAW": 126672, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22800.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30534.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133701, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_915408,ChtcWrapper66.out,AuditLog.66,simu_3_66.txt,harvest.log,66.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110792, "ExitBySignal": false, "LastMatchTime": 1446110790, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22911, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/66/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22911.0d, "LastJobLeaseRenewal": 1446133701, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "66+66", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e455.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 81.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/66/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133701, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582421, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22911.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=66 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.244.247:44193>#1444685638#5766#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/66", "QDate": 1446105690, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582432.0#1446105695", "StatsLifetimeStarter": 22249, "JobStartDate": 1446110791, "SubmitEventNotes": "DAG Node: 39+39", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.244.247", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110791, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22250.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133041, "ResidentSetSize_RAW": 125680, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22111.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27942.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133041, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_915422,ChtcWrapper39.out,AuditLog.39,simu_3_39.txt,harvest.log,39.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110792, "ExitBySignal": false, "LastMatchTime": 1446110791, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22250, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/39/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22250.0d, "LastJobLeaseRenewal": 1446133041, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "39+39", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e455.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 119.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/39/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133041, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582432, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22250.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=39 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.244.247:44193>#1444685638#5772#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/39", "QDate": 1446105695, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582440.0#1446105697", "StatsLifetimeStarter": 22526, "JobStartDate": 1446110791, "SubmitEventNotes": "DAG Node: 382+382", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.123", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110791, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22528.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133319, "ResidentSetSize_RAW": 125040, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22357.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30506.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133319, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2233833,ChtcWrapper382.out,AuditLog.382,simu_3_382.txt,harvest.log,382.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110793, "ExitBySignal": false, "LastMatchTime": 1446110791, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22528, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/382/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22528.0d, "LastJobLeaseRenewal": 1446133319, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "382+382", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e323.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 135.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/382/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133319, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582440, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22528.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=382 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.123:60803>#1444759965#6770#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/382", "QDate": 1446105697, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582460.0#1446105706", "StatsLifetimeStarter": 22393, "JobStartDate": 1446111074, "SubmitEventNotes": "DAG Node: 58+58", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.184", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111074, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22394.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133468, "ResidentSetSize_RAW": 127308, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22278.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30499.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133468, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2507136,ChtcWrapper58.out,AuditLog.58,simu_3_58.txt,harvest.log,58.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111075, "ExitBySignal": false, "LastMatchTime": 1446111074, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22394, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/58/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22394.0d, "LastJobLeaseRenewal": 1446133468, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "58+58", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e384.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 81.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/58/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133468, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582460, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22394.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=58 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.184:62907>#1443991428#13854#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/58", "QDate": 1446105706, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582462.0#1446105706", "StatsLifetimeStarter": 22519, "JobStartDate": 1446111074, "SubmitEventNotes": "DAG Node: 86+86", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.185", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111074, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22520.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133594, "ResidentSetSize_RAW": 125772, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22371.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30498.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133594, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3353135,ChtcWrapper86.out,AuditLog.86,simu_3_86.txt,harvest.log,86.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111075, "ExitBySignal": false, "LastMatchTime": 1446111074, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22520, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/86/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22520.0d, "LastJobLeaseRenewal": 1446133594, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "86+86", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e385.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 120.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/86/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133594, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582462, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22520.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=86 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.185:43838>#1443991427#12472#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/86", "QDate": 1446105706, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582463.0#1446105706", "StatsLifetimeStarter": 21403, "JobStartDate": 1446111073, "SubmitEventNotes": "DAG Node: 77+77", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.131", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111073, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 21404.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132477, "ResidentSetSize_RAW": 124948, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21243.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27912.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132477, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1137029,ChtcWrapper77.out,AuditLog.77,simu_3_77.txt,harvest.log,77.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111074, "ExitBySignal": false, "LastMatchTime": 1446111073, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21404, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/77/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21404.0d, "LastJobLeaseRenewal": 1446132477, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "77+77", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e331.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 134.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/77/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132477, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582463, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21404.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=77 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.131:14956>#1444819395#7764#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/77", "QDate": 1446105706, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582467.0#1446105708", "StatsLifetimeStarter": 22208, "JobStartDate": 1446111073, "SubmitEventNotes": "DAG Node: 275+275", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.87", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111073, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22209.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133282, "ResidentSetSize_RAW": 126040, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22110.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30485.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133282, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_626043,ChtcWrapper275.out,AuditLog.275,simu_3_275.txt,harvest.log,275.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111074, "ExitBySignal": false, "LastMatchTime": 1446111073, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22209, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/275/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22209.0d, "LastJobLeaseRenewal": 1446133282, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "275+275", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e287.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 78.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/275/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133282, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582467, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22209.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=275 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.87:2102>#1444759894#8469#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/275", "QDate": 1446105708, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582476.0#1446105711", "StatsLifetimeStarter": 22252, "JobStartDate": 1446111073, "SubmitEventNotes": "DAG Node: 96+96", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.127", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111073, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 22253.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133326, "ResidentSetSize_RAW": 127304, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22096.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28231.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133328, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2637948,ChtcWrapper96.out,AuditLog.96,simu_3_96.txt,harvest.log,96.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111074, "ExitBySignal": false, "LastMatchTime": 1446111073, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22253, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/96/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22253.0d, "LastJobLeaseRenewal": 1446133326, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "96+96", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e327.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 124.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/96/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133326, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582476, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22253.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=96 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.127:48134>#1443991405#8554#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/96", "QDate": 1446105711, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582487.0#1446105716", "StatsLifetimeStarter": 22573, "JobStartDate": 1446111074, "SubmitEventNotes": "DAG Node: 87+87", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.127", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111074, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22575.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133649, "ResidentSetSize_RAW": 128908, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22397.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27905.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133649, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2637965,ChtcWrapper87.out,AuditLog.87,simu_3_87.txt,harvest.log,87.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111076, "ExitBySignal": false, "LastMatchTime": 1446111074, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22575, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/87/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22575.0d, "LastJobLeaseRenewal": 1446133649, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "87+87", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e327.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 125.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/87/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133649, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582487, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22575.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=87 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.127:48134>#1443991405#8558#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/87", "QDate": 1446105716, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582489.0#1446105717", "StatsLifetimeStarter": 22699, "JobStartDate": 1446111074, "SubmitEventNotes": "DAG Node: 69+69", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.237", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111074, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22702.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133776, "ResidentSetSize_RAW": 126444, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22552.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30552.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133776, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1529941,ChtcWrapper69.out,AuditLog.69,simu_3_69.txt,harvest.log,69.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111076, "ExitBySignal": false, "LastMatchTime": 1446111074, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22702, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/69/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22702.0d, "LastJobLeaseRenewal": 1446133776, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "69+69", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e437.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 122.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/69/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133776, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582489, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22702.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=69 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.237:1373>#1444673410#8302#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/69", "QDate": 1446105717, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582491.0#1446105717", "StatsLifetimeStarter": 22279, "JobStartDate": 1446111075, "SubmitEventNotes": "DAG Node: 88+88", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.184", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111075, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 22280.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133355, "ResidentSetSize_RAW": 126084, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22117.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27905.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133355, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2507151,ChtcWrapper88.out,AuditLog.88,simu_3_88.txt,harvest.log,88.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111076, "ExitBySignal": false, "LastMatchTime": 1446111075, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22280, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/88/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22280.0d, "LastJobLeaseRenewal": 1446133355, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "88+88", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e384.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 144.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/88/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133355, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582491, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22280.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=88 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.184:62907>#1443991428#13858#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/88", "QDate": 1446105717, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582494.0#1446105717", "StatsLifetimeStarter": 21401, "JobStartDate": 1446111075, "SubmitEventNotes": "DAG Node: 89+89", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.97", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111075, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 21403.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132478, "ResidentSetSize_RAW": 126072, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21263.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30508.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132478, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1266453,ChtcWrapper89.out,AuditLog.89,simu_3_89.txt,harvest.log,89.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111077, "ExitBySignal": false, "LastMatchTime": 1446111075, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21403, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/89/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21403.0d, "LastJobLeaseRenewal": 1446132478, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "89+89", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e297.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 114.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/89/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132478, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582494, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21403.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=89 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.97:50007>#1444685419#10092#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/89", "QDate": 1446105717, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582510.0#1446105723", "StatsLifetimeStarter": 21254, "JobStartDate": 1446111276, "SubmitEventNotes": "DAG Node: 210+210", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.66", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111276, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 100000, "RemoteWallClockTime": 21256.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 4, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132532, "ResidentSetSize_RAW": 75336, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21046.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 122840, "BytesSent": 30503.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446132532, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 540, "SpooledOutputFiles": "CURLTIME_1268212,ChtcWrapper210.out,AuditLog.210,simu_3_210.txt,harvest.log,210.out", "BlockWriteKbytes": 4, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111278, "ExitBySignal": false, "LastMatchTime": 1446111276, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 1407, "DAGManJobId": 49582200, "MemoryUsage": 97, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 51368, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21256, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/210/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21256.0d, "LastJobLeaseRenewal": 1446132532, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "210+210", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c047.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 169.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/210/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132532, "StreamErr": false, "RecentBlockReadKbytes": 8940, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582510, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21256.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=210 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.66:54632>#1445311857#1452#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/210", "QDate": 1446105723, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582511.0#1446105723", "StatsLifetimeStarter": 22242, "JobStartDate": 1446111276, "SubmitEventNotes": "DAG Node: 201+201", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 13, "StartdPrincipal": "execute-side@matchsession/128.104.55.68", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111276, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 75000, "RemoteWallClockTime": 22243.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 4, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133519, "ResidentSetSize_RAW": 72052, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22043.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 117304, "BytesSent": 30483.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446133520, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 460, "SpooledOutputFiles": "CURLTIME_935737,ChtcWrapper201.out,AuditLog.201,simu_3_201.txt,harvest.log,201.out", "BlockWriteKbytes": 4, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111277, "ExitBySignal": false, "LastMatchTime": 1446111276, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 1906, "DAGManJobId": 49582200, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 81056, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22243, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/201/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22243.0d, "LastJobLeaseRenewal": 1446133519, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "201+201", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c049.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 133.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/201/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133519, "StreamErr": false, "RecentBlockReadKbytes": 4868, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582511, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22243.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=201 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.68:4958>#1445345121#1580#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/201", "QDate": 1446105723, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582525.0#1446105728", "StatsLifetimeStarter": 22429, "JobStartDate": 1446111452, "SubmitEventNotes": "DAG Node: 300+300", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.126", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111452, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22430.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133882, "ResidentSetSize_RAW": 126740, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22300.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133883, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1700927,ChtcWrapper300.out,AuditLog.300,simu_3_300.txt,harvest.log,300.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111453, "ExitBySignal": false, "LastMatchTime": 1446111452, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22430, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/300/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22430.0d, "LastJobLeaseRenewal": 1446133882, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "300+300", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e326.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 112.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/300/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133882, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582525, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22430.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=300 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.126:40098>#1444759970#7928#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/300", "QDate": 1446105728, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582533.0#1446105734", "StatsLifetimeStarter": 22519, "JobStartDate": 1446111647, "SubmitEventNotes": "DAG Node: 211+211", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.61", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111647, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22520.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134167, "ResidentSetSize_RAW": 126608, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22353.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30603.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134167, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_818403,ChtcWrapper211.out,AuditLog.211,simu_3_211.txt,harvest.log,211.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111648, "ExitBySignal": false, "LastMatchTime": 1446111647, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22520, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/211/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22520.0d, "LastJobLeaseRenewal": 1446134167, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "211+211", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e261.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 137.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/211/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134167, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582533, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22520.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=211 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.61:49736>#1444759807#6759#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/211", "QDate": 1446105734, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582539.0#1446105734", "StatsLifetimeStarter": 21532, "JobStartDate": 1446111811, "SubmitEventNotes": "DAG Node: 121+121", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.102", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111811, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 21534.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133345, "ResidentSetSize_RAW": 125956, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21392.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30557.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133345, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3934729,ChtcWrapper121.out,AuditLog.121,simu_3_121.txt,harvest.log,121.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111812, "ExitBySignal": false, "LastMatchTime": 1446111811, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21534, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/121/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21534.0d, "LastJobLeaseRenewal": 1446133345, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "121+121", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e302.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 122.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/121/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133345, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582539, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21534.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=121 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.102:26944>#1443991374#13421#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/121", "QDate": 1446105734, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582540.0#1446105734", "StatsLifetimeStarter": 22050, "JobStartDate": 1446111810, "SubmitEventNotes": "DAG Node: 220+220", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.126", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111810, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 22052.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133862, "ResidentSetSize_RAW": 126940, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21897.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28344.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133862, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1702594,ChtcWrapper220.out,AuditLog.220,simu_3_220.txt,harvest.log,220.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111812, "ExitBySignal": false, "LastMatchTime": 1446111810, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22052, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/220/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22052.0d, "LastJobLeaseRenewal": 1446133862, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "220+220", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e326.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 127.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/220/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133862, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582540, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22052.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=220 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.126:40098>#1444759970#7932#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/220", "QDate": 1446105734, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582553.0#1446105740", "StatsLifetimeStarter": 21632, "JobStartDate": 1446111993, "SubmitEventNotes": "DAG Node: 121+121", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.141", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111993, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 21635.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133628, "ResidentSetSize_RAW": 126224, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21477.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30505.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133628, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2585208,ChtcWrapper121.out,AuditLog.121,simu_3_121.txt,harvest.log,121.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111995, "ExitBySignal": false, "LastMatchTime": 1446111993, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21635, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/121/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21635.0d, "LastJobLeaseRenewal": 1446133628, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "121+121", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e341.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 134.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/121/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133628, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582553, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21635.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=121 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.141:7534>#1444673425#9485#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/121", "QDate": 1446105740, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582557.0#1446105741", "StatsLifetimeStarter": 21953, "JobStartDate": 1446112222, "SubmitEventNotes": "DAG Node: 159+159", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.152", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446112222, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 21954.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134176, "ResidentSetSize_RAW": 125604, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21791.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30561.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134177, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2696692,ChtcWrapper159.out,AuditLog.159,simu_3_159.txt,harvest.log,159.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446112223, "ExitBySignal": false, "LastMatchTime": 1446112222, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 147, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21954, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/159/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21954.0d, "LastJobLeaseRenewal": 1446134176, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "159+159", "PeriodicRelease": false, "JobRunCount": 1, "LastRemoteHost": "slot1@e352.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 137.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/159/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134176, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": false, "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582557, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21954.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=159 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.152:39021>#1444772294#9281#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/159", "QDate": 1446105741, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582585.0#1446105751", "StatsLifetimeStarter": 20690, "JobStartDate": 1446112544, "SubmitEventNotes": "DAG Node: 311+311", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.244", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446112544, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 20692.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133236, "ResidentSetSize_RAW": 126832, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 20568.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27894.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133236, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_363317,ChtcWrapper311.out,AuditLog.311,simu_3_311.txt,harvest.log,311.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446112547, "ExitBySignal": false, "LastMatchTime": 1446112544, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 20692, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/311/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 20692.0d, "LastJobLeaseRenewal": 1446133236, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "311+311", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e444.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 87.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/311/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133236, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582585, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 20692.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=311 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.244:1411>#1443991446#13076#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/311", "QDate": 1446105751, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582591.0#1446105753", "StatsLifetimeStarter": 19824, "JobStartDate": 1446112695, "SubmitEventNotes": "DAG Node: 438+438", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.211", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446112695, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 19825.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132520, "ResidentSetSize_RAW": 125924, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 19694.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 29426.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132520, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1907890,ChtcWrapper438.out,AuditLog.438,simu_3_438.txt,harvest.log,438.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446112696, "ExitBySignal": false, "LastMatchTime": 1446112695, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 19825, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/438/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 19825.0d, "LastJobLeaseRenewal": 1446132520, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "438+438", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e411.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 107.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/438/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132520, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582591, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 19825.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=438 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.211:65149>#1443991444#12482#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/438", "QDate": 1446105753, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582595.0#1446105757", "StatsLifetimeStarter": 20366, "JobStartDate": 1446112695, "SubmitEventNotes": "DAG Node: 104+104", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.193", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446112695, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 20367.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133062, "ResidentSetSize_RAW": 125640, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 20221.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31674.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133062, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_173815,ChtcWrapper104.out,AuditLog.104,simu_3_104.txt,harvest.log,104.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446112696, "ExitBySignal": false, "LastMatchTime": 1446112695, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 20367, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/104/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 20367.0d, "LastJobLeaseRenewal": 1446133062, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "104+104", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e393.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 126.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/104/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133062, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582595, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 20367.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=104 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.193:65434>#1443991433#12882#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/104", "QDate": 1446105757, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582601.0#1446105757", "StatsLifetimeStarter": 19359, "JobStartDate": 1446113038, "SubmitEventNotes": "DAG Node: 231+231", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.197", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446113038, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 19360.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132398, "ResidentSetSize_RAW": 125720, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 19226.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30507.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132398, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3355874,ChtcWrapper231.out,AuditLog.231,simu_3_231.txt,harvest.log,231.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446113039, "ExitBySignal": false, "LastMatchTime": 1446113038, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 19360, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/231/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 19360.0d, "LastJobLeaseRenewal": 1446132398, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "231+231", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e397.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 111.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/231/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132398, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582601, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 19360.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=231 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.197:37993>#1443991431#13795#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/231", "QDate": 1446105757, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582610.0#1446105762", "StatsLifetimeStarter": 20371, "JobStartDate": 1446113038, "SubmitEventNotes": "DAG Node: 213+213", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.219", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446113038, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 20372.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133410, "ResidentSetSize_RAW": 124944, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 20214.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30507.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133410, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2647255,ChtcWrapper213.out,AuditLog.213,simu_3_213.txt,harvest.log,213.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446113039, "ExitBySignal": false, "LastMatchTime": 1446113038, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 20372, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/213/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 20372.0d, "LastJobLeaseRenewal": 1446133410, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "213+213", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e419.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 132.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/213/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133410, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582610, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 20372.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=213 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.219:51004>#1443991439#14297#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/213", "QDate": 1446105762, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582641.0#1446105773", "StatsLifetimeStarter": 18408, "JobStartDate": 1446114160, "SubmitEventNotes": "DAG Node: 105+105", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.166", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446114160, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 18410.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132570, "ResidentSetSize_RAW": 124020, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18271.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28241.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132570, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1464388,ChtcWrapper105.out,AuditLog.105,simu_3_105.txt,harvest.log,105.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446114162, "ExitBySignal": false, "LastMatchTime": 1446114160, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18410, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/105/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18410.0d, "LastJobLeaseRenewal": 1446132570, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "105+105", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e366.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 118.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/105/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132570, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582641, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18410.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=105 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.166:20019>#1444831317#8851#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/105", "QDate": 1446105773, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582659.0#1446105779", "StatsLifetimeStarter": 19336, "JobStartDate": 1446114724, "SubmitEventNotes": "DAG Node: 232+232", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.48", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446114724, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 75000, "RemoteWallClockTime": 19338.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216668, "EnteredCurrentStatus": 1446134062, "ResidentSetSize_RAW": 71268, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 19081.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 118772, "BytesSent": 27911.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134062, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 3, "SpooledOutputFiles": "harvest.log,232.out,ChtcWrapper232.out,AuditLog.232,CURLTIME_1864147,simu_3_232.txt", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446114726, "ExitBySignal": false, "LastMatchTime": 1446114724, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 615, "DAGManJobId": 49582200, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 26436, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 19338, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/232/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 19338.0d, "LastJobLeaseRenewal": 1446134062, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "232+232", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c029.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 179.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/232/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134062, "StreamErr": false, "RecentBlockReadKbytes": 12, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582659, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 19338.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=232 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.48:26476>#1445344800#1612#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/232", "QDate": 1446105779, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582699.0#1446105797", "StatsLifetimeStarter": 18984, "JobStartDate": 1446114861, "SubmitEventNotes": "DAG Node: 313+313", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.157", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446114861, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 18985.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133846, "ResidentSetSize_RAW": 126756, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18847.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27896.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133846, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_930697,ChtcWrapper313.out,AuditLog.313,simu_3_313.txt,harvest.log,313.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446114862, "ExitBySignal": false, "LastMatchTime": 1446114861, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18985, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/313/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18985.0d, "LastJobLeaseRenewal": 1446133846, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "313+313", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e357.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 107.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/313/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133846, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582699, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18985.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=313 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.157:33109>#1444685526#8861#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/313", "QDate": 1446105797, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582722.0#1446105803", "StatsLifetimeStarter": 18066, "JobStartDate": 1446115114, "SubmitEventNotes": "DAG Node: 298+298", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.197", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115114, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 18067.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133181, "ResidentSetSize_RAW": 127108, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17941.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31693.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133181, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3362144,ChtcWrapper298.out,AuditLog.298,simu_3_298.txt,harvest.log,298.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115115, "ExitBySignal": false, "LastMatchTime": 1446115114, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18067, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/298/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18067.0d, "LastJobLeaseRenewal": 1446133181, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "298+298", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e397.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 110.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/298/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133181, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582722, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18067.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=298 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.197:37993>#1443991431#13812#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/298", "QDate": 1446105803, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582724.0#1446105803", "StatsLifetimeStarter": 18902, "JobStartDate": 1446115114, "SubmitEventNotes": "DAG Node: 260+260", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.164", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115114, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 18903.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134017, "ResidentSetSize_RAW": 124924, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18782.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30507.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134017, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2890029,ChtcWrapper260.out,AuditLog.260,simu_3_260.txt,harvest.log,260.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115115, "ExitBySignal": false, "LastMatchTime": 1446115114, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18903, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/260/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18903.0d, "LastJobLeaseRenewal": 1446134017, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "260+260", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e364.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 109.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/260/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134017, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582724, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18903.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=260 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.164:7769>#1444760010#7999#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/260", "QDate": 1446105803, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582760.0#1446105819", "StatsLifetimeStarter": 18410, "JobStartDate": 1446115399, "SubmitEventNotes": "DAG Node: 170+170", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.113", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115399, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 18411.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133810, "ResidentSetSize_RAW": 125400, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18266.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28239.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133810, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2614862,ChtcWrapper170.out,AuditLog.170,simu_3_170.txt,harvest.log,170.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115400, "ExitBySignal": false, "LastMatchTime": 1446115399, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18411, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/170/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18411.0d, "LastJobLeaseRenewal": 1446133810, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "170+170", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e313.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 104.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/170/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133810, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582760, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18411.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=170 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.113:56191>#1443991385#10335#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/170", "QDate": 1446105819, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582771.0#1446105825", "StatsLifetimeStarter": 17065, "JobStartDate": 1446115399, "SubmitEventNotes": "DAG Node: 350+350", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.197", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115399, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 17066.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132465, "ResidentSetSize_RAW": 126136, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16934.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132465, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3362491,ChtcWrapper350.out,AuditLog.350,simu_3_350.txt,harvest.log,350.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115400, "ExitBySignal": false, "LastMatchTime": 1446115399, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17066, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/350/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17066.0d, "LastJobLeaseRenewal": 1446132465, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "350+350", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e397.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 103.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/350/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132465, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582771, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17066.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=350 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.197:37993>#1443991431#13821#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/350", "QDate": 1446105825, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582782.0#1446105831", "StatsLifetimeStarter": 18187, "JobStartDate": 1446115778, "SubmitEventNotes": "DAG Node: 440+440", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.158", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115778, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 18189.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133967, "ResidentSetSize_RAW": 125632, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18054.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27914.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133967, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1401618,ChtcWrapper440.out,AuditLog.440,simu_3_440.txt,harvest.log,440.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115779, "ExitBySignal": false, "LastMatchTime": 1446115778, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18189, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/440/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18189.0d, "LastJobLeaseRenewal": 1446133967, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "440+440", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e358.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 105.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/440/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133967, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582782, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18189.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=440 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.158:24962>#1444759998#9425#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/440", "QDate": 1446105831, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582783.0#1446105831", "StatsLifetimeStarter": 18022, "JobStartDate": 1446115778, "SubmitEventNotes": "DAG Node: 270+270", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.242", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115778, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 18023.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133801, "ResidentSetSize_RAW": 127404, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17875.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27877.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133801, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_24492,ChtcWrapper270.out,AuditLog.270,simu_3_270.txt,harvest.log,270.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115779, "ExitBySignal": false, "LastMatchTime": 1446115778, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18023, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/270/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18023.0d, "LastJobLeaseRenewal": 1446133801, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "270+270", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e442.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 115.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/270/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133801, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582783, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18023.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=270 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.242:38884>#1443991450#10410#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/270", "QDate": 1446105831, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582786.0#1446105835", "StatsLifetimeStarter": 18247, "JobStartDate": 1446115778, "SubmitEventNotes": "DAG Node: 3+3", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.107", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115778, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 18248.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134026, "ResidentSetSize_RAW": 125940, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18118.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27896.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134026, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3140097,ChtcWrapper3.out,AuditLog.3,simu_3_3.txt,harvest.log,3.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115779, "ExitBySignal": false, "LastMatchTime": 1446115778, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18248, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/3/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18248.0d, "LastJobLeaseRenewal": 1446134026, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284717.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "3+3", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e307.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 108.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/3/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134026, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582786, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18248.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=3 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.107:63744>#1444685448#11070#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/3", "QDate": 1446105835, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582794.0#1446105836", "StatsLifetimeStarter": 17555, "JobStartDate": 1446115778, "SubmitEventNotes": "DAG Node: 252+252", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.174", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115778, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 17557.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133335, "ResidentSetSize_RAW": 126656, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17422.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27913.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133335, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3591632,ChtcWrapper252.out,AuditLog.252,simu_3_252.txt,harvest.log,252.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115780, "ExitBySignal": false, "LastMatchTime": 1446115778, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17557, "ExecutableSize_RAW": 6, "LastRejMatchReason": "no match found", "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/252/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17557.0d, "LastJobLeaseRenewal": 1446133335, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "252+252", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e374.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 108.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/252/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133335, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "LastRejMatchTime": 1446115777, "JobLeaseDuration": 2400, "ClusterId": 49582794, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17557.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=252 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.174:7981>#1444760024#8714#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/252", "QDate": 1446105836, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582796.0#1446105836", "StatsLifetimeStarter": 17251, "JobStartDate": 1446115778, "SubmitEventNotes": "DAG Node: 243+243", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.225", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115778, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 17253.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133031, "ResidentSetSize_RAW": 127320, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17159.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27913.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133031, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2889473,ChtcWrapper243.out,AuditLog.243,simu_3_243.txt,harvest.log,243.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115780, "ExitBySignal": false, "LastMatchTime": 1446115778, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17253, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/243/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17253.0d, "LastJobLeaseRenewal": 1446133031, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "243+243", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e425.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 62.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/243/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133031, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582796, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17253.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=243 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.225:15992>#1444673418#9764#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/243", "QDate": 1446105836, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582827.0#1446105847", "StatsLifetimeStarter": 16585, "JobStartDate": 1446115924, "SubmitEventNotes": "DAG Node: 162+162", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.161", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115924, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 16587.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132511, "ResidentSetSize_RAW": 126964, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16462.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30506.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132511, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2195199,ChtcWrapper162.out,AuditLog.162,simu_3_162.txt,harvest.log,162.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115926, "ExitBySignal": false, "LastMatchTime": 1446115924, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16587, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/162/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16587.0d, "LastJobLeaseRenewal": 1446132511, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "162+162", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e361.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 94.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/162/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132511, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582827, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16587.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=162 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.161:7475>#1443991415#14144#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/162", "QDate": 1446105847, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582831.0#1446105852", "StatsLifetimeStarter": 17603, "JobStartDate": 1446115924, "SubmitEventNotes": "DAG Node: 31+31", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.65", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115924, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 17605.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133529, "ResidentSetSize_RAW": 124912, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17471.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27905.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133529, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3813186,ChtcWrapper31.out,AuditLog.31,simu_3_31.txt,harvest.log,31.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115925, "ExitBySignal": false, "LastMatchTime": 1446115924, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17604, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/31/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17605.0d, "LastJobLeaseRenewal": 1446133528, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284718.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "31+31", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e265.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 101.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/31/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133529, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582831, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17604.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=31 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.65:22193>#1444759815#9517#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/31", "QDate": 1446105852, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582862.0#1446105863", "StatsLifetimeStarter": 16713, "JobStartDate": 1446116497, "SubmitEventNotes": "DAG Node: 51+51", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.151", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116497, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 16714.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133211, "ResidentSetSize_RAW": 124628, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16576.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30499.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133211, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1045787,ChtcWrapper51.out,AuditLog.51,simu_3_51.txt,harvest.log,51.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116498, "ExitBySignal": false, "LastMatchTime": 1446116497, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16714, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/51/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16714.0d, "LastJobLeaseRenewal": 1446133211, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284718.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "51+51", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e351.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 103.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/51/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133211, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582862, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16714.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=51 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.151:14279>#1445444483#5155#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/51", "QDate": 1446105863, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582867.0#1446105863", "StatsLifetimeStarter": 16609, "JobStartDate": 1446116497, "SubmitEventNotes": "DAG Node: 423+423", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.75", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116497, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 16610.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133107, "ResidentSetSize_RAW": 127232, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16440.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28235.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133107, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3323529,ChtcWrapper423.out,AuditLog.423,simu_3_423.txt,harvest.log,423.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116498, "ExitBySignal": false, "LastMatchTime": 1446116497, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16610, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/423/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16610.0d, "LastJobLeaseRenewal": 1446133107, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "423+423", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e275.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 139.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/423/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133107, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582867, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16610.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=423 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.75:36755>#1444759846#8549#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/423", "QDate": 1446105863, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582873.0#1446105864", "StatsLifetimeStarter": 17505, "JobStartDate": 1446116497, "SubmitEventNotes": "DAG Node: 280+280", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.244", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116497, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 17506.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134003, "ResidentSetSize_RAW": 126200, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17357.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27894.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134003, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_373303,ChtcWrapper280.out,AuditLog.280,simu_3_280.txt,harvest.log,280.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116498, "ExitBySignal": false, "LastMatchTime": 1446116497, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17506, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/280/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17506.0d, "LastJobLeaseRenewal": 1446134003, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "280+280", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e444.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 112.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/280/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134003, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582873, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17506.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=280 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.244:1411>#1443991446#13109#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/280", "QDate": 1446105864, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582874.0#1446105868", "StatsLifetimeStarter": 15928, "JobStartDate": 1446116497, "SubmitEventNotes": "DAG Node: 70+70", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.194", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116497, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 15929.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132426, "ResidentSetSize_RAW": 126992, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 15804.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27908.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132426, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3390859,ChtcWrapper70.out,AuditLog.70,simu_3_70.txt,harvest.log,70.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116498, "ExitBySignal": false, "LastMatchTime": 1446116497, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 15929, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/70/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 15929.0d, "LastJobLeaseRenewal": 1446132426, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284718.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "70+70", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e394.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 100.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/70/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132426, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582874, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 15929.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=70 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.194:52833>#1443991432#16216#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/70", "QDate": 1446105868, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582878.0#1446105869", "StatsLifetimeStarter": 17433, "JobStartDate": 1446116497, "SubmitEventNotes": "DAG Node: 43+43", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.244", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116497, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 17435.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133932, "ResidentSetSize_RAW": 124328, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17306.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30479.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133932, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_373315,ChtcWrapper43.out,AuditLog.43,simu_3_43.txt,harvest.log,43.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116499, "ExitBySignal": false, "LastMatchTime": 1446116497, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17435, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/43/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17435.0d, "LastJobLeaseRenewal": 1446133932, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284718.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "43+43", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e444.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 106.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/43/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133932, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582878, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17435.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=43 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.244:1411>#1443991446#13114#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/43", "QDate": 1446105869, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582883.0#1446105869", "StatsLifetimeStarter": 16052, "JobStartDate": 1446116623, "SubmitEventNotes": "DAG Node: 450+450", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.188", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116623, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 16053.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132676, "ResidentSetSize_RAW": 125676, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 15939.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30507.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132676, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2688575,ChtcWrapper450.out,AuditLog.450,simu_3_450.txt,harvest.log,450.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116624, "ExitBySignal": false, "LastMatchTime": 1446116623, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16053, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/450/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16053.0d, "LastJobLeaseRenewal": 1446132676, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "450+450", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e388.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 86.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/450/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132676, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582883, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16053.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=450 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.188:28065>#1443991430#12995#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/450", "QDate": 1446105869, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582897.0#1446105875", "StatsLifetimeStarter": 16343, "JobStartDate": 1446116750, "SubmitEventNotes": "DAG Node: 226+226", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.235", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116750, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 16344.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133094, "ResidentSetSize_RAW": 124920, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16201.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133094, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3306268,ChtcWrapper226.out,AuditLog.226,simu_3_226.txt,harvest.log,226.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116751, "ExitBySignal": false, "LastMatchTime": 1446116750, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16344, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/226/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16344.0d, "LastJobLeaseRenewal": 1446133094, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "226+226", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e435.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 119.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/226/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133094, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582897, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16344.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=226 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.235:26914>#1443991459#10913#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/226", "QDate": 1446105875, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582902.0#1446105875", "StatsLifetimeStarter": 16445, "JobStartDate": 1446116750, "SubmitEventNotes": "DAG Node: 136+136", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.194", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116750, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 16446.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133196, "ResidentSetSize_RAW": 126576, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16315.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30507.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133196, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3391975,ChtcWrapper136.out,AuditLog.136,simu_3_136.txt,harvest.log,136.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116751, "ExitBySignal": false, "LastMatchTime": 1446116750, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16446, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/136/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16446.0d, "LastJobLeaseRenewal": 1446133196, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "136+136", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e394.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 106.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/136/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133196, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582902, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16446.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=136 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.194:52833>#1443991432#16220#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/136", "QDate": 1446105875, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583239.0#1446106003", "StatsLifetimeStarter": 13050, "JobStartDate": 1446121053, "SubmitEventNotes": "DAG Node: 409+409", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.242", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121053, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 13051.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134104, "ResidentSetSize_RAW": 127216, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12934.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27873.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134104, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_37424,ChtcWrapper409.out,AuditLog.409,simu_3_409.txt,harvest.log,409.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121054, "ExitBySignal": false, "LastMatchTime": 1446121053, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 13051, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/409/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 13051.0d, "LastJobLeaseRenewal": 1446134104, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "409+409", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e442.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 93.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/409/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134104, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583239, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 13051.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=409 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.242:38884>#1443991450#10456#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/409", "QDate": 1446106003, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583254.0#1446106008", "StatsLifetimeStarter": 12361, "JobStartDate": 1446121052, "SubmitEventNotes": "DAG Node: 275+275", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.163", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121052, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 12363.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133415, "ResidentSetSize_RAW": 126732, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12249.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30506.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133415, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2948896,ChtcWrapper275.out,AuditLog.275,simu_3_275.txt,harvest.log,275.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121054, "ExitBySignal": false, "LastMatchTime": 1446121052, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 12363, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/275/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 12363.0d, "LastJobLeaseRenewal": 1446133415, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "275+275", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e363.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 95.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/275/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133415, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583254, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 12363.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=275 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.163:21972>#1443991420#10986#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/275", "QDate": 1446106008, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583271.0#1446106014", "StatsLifetimeStarter": 11405, "JobStartDate": 1446121053, "SubmitEventNotes": "DAG Node: 149+149", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.101.145", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121053, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 11407.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132460, "ResidentSetSize_RAW": 124492, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 11060.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132460, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3185190,ChtcWrapper149.out,AuditLog.149,simu_3_149.txt,harvest.log,149.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121055, "ExitBySignal": false, "LastMatchTime": 1446121053, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 11407, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/149/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 11407.0d, "LastJobLeaseRenewal": 1446132460, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "149+149", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e145.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 70.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/149/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132460, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583271, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 11407.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=149 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.101.145:57668>#1444053387#12271#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/149", "QDate": 1446106014, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583278.0#1446106014", "StatsLifetimeStarter": 12654, "JobStartDate": 1446121054, "SubmitEventNotes": "DAG Node: 239+239", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.140", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121054, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 12656.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133710, "ResidentSetSize_RAW": 124780, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12549.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28220.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133710, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_548136,ChtcWrapper239.out,AuditLog.239,simu_3_239.txt,harvest.log,239.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121056, "ExitBySignal": false, "LastMatchTime": 1446121054, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 12656, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/239/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 12656.0d, "LastJobLeaseRenewal": 1446133710, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "239+239", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e340.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 88.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/239/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133710, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583278, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 12656.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=239 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.140:58412>#1444681013#9585#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/239", "QDate": 1446106014, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583285.0#1446106020", "StatsLifetimeStarter": 12499, "JobStartDate": 1446121054, "SubmitEventNotes": "DAG Node: 194+194", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.73", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121054, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 12501.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133555, "ResidentSetSize_RAW": 125892, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12398.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133555, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2796351,ChtcWrapper194.out,AuditLog.194,simu_3_194.txt,harvest.log,194.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121056, "ExitBySignal": false, "LastMatchTime": 1446121054, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 12501, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/194/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 12501.0d, "LastJobLeaseRenewal": 1446133555, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "194+194", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e273.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 87.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/194/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133555, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583285, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 12501.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=194 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.73:33900>#1444759838#9136#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/194", "QDate": 1446106019, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583286.0#1446106020", "StatsLifetimeStarter": 11906, "JobStartDate": 1446121054, "SubmitEventNotes": "DAG Node: 284+284", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.101.145", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121054, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 11908.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132962, "ResidentSetSize_RAW": 122612, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 11624.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132962, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3185196,ChtcWrapper284.out,AuditLog.284,simu_3_284.txt,harvest.log,284.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121056, "ExitBySignal": false, "LastMatchTime": 1446121054, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 11908, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/284/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 11908.0d, "LastJobLeaseRenewal": 1446132962, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "284+284", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e145.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 73.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/284/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132962, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583286, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 11908.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=284 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.101.145:57668>#1444053387#12274#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/284", "QDate": 1446106020, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583295.0#1446106024", "StatsLifetimeStarter": 12273, "JobStartDate": 1446121055, "SubmitEventNotes": "DAG Node: 421+421", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.73", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121055, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 12274.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133329, "ResidentSetSize_RAW": 127332, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12186.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133329, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2796365,ChtcWrapper421.out,AuditLog.421,simu_3_421.txt,harvest.log,421.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121056, "ExitBySignal": false, "LastMatchTime": 1446121055, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 12274, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/421/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 12274.0d, "LastJobLeaseRenewal": 1446133329, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284719.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "421+421", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e273.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 70.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/421/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133329, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583295, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 12274.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=421 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.73:33900>#1444759838#9139#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/421", "QDate": 1446106024, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583316.0#1446106031", "StatsLifetimeStarter": 12602, "JobStartDate": 1446121412, "SubmitEventNotes": "DAG Node: 419+419", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.140", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121412, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 12604.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134016, "ResidentSetSize_RAW": 125420, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12491.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30485.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134016, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_549258,ChtcWrapper419.out,AuditLog.419,simu_3_419.txt,harvest.log,419.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121413, "ExitBySignal": false, "LastMatchTime": 1446121412, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 12604, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/419/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 12604.0d, "LastJobLeaseRenewal": 1446134016, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "419+419", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e340.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 86.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/419/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134016, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583316, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 12604.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=419 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.140:58412>#1444681013#9588#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/419", "QDate": 1446106031, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583329.0#1446106036", "StatsLifetimeStarter": 11940, "JobStartDate": 1446121413, "SubmitEventNotes": "DAG Node: 392+392", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.101.129", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121413, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 11942.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133355, "ResidentSetSize_RAW": 119932, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 11714.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30504.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133355, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1549103,ChtcWrapper392.out,AuditLog.392,simu_3_392.txt,harvest.log,392.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121414, "ExitBySignal": false, "LastMatchTime": 1446121413, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 11942, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/392/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 11942.0d, "LastJobLeaseRenewal": 1446133355, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "392+392", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e129.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 81.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/392/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133355, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583329, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 11942.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=392 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.101.129:24642>#1444053399#13743#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/392", "QDate": 1446106036, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583811.0#1446133780", "StatsLifetimeStarter": 39, "JobStartDate": 1446133930, "SubmitEventNotes": "DAG Node: 01100+01100", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38254, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.105.245.6", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133930, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 7500, "RemoteWallClockTime": 40.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1206320, "EnteredCurrentStatus": 1446133970, "ResidentSetSize_RAW": 5044, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 10.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5048, "BytesSent": 2727275.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133971, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 8, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_2893592,ChtcWrapper01100.out,R2011b_INFO,AuditLog.01100,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133931, "ExitBySignal": false, "LastMatchTime": 1446133930, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 8, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 108, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 40, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/01100/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 40.0d, "LastJobLeaseRenewal": 1446133970, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "01100+01100", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e206.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 6.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/01100/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133970, "StreamErr": false, "RecentBlockReadKbytes": 108, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583811, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 30, "CommittedSlotTime": 40.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=01100 --", "Environment": "", "LastPublicClaimId": "<128.105.245.6:9783>#1444977535#2490#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/01100", "QDate": 1446133780, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583854.0#1446133826", "StatsLifetimeStarter": 46, "JobStartDate": 1446133932, "SubmitEventNotes": "DAG Node: 10200+10200", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38256, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.104.58.32", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133932, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 5000, "RemoteWallClockTime": 48.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1206292, "EnteredCurrentStatus": 1446133980, "ResidentSetSize_RAW": 4816, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 15.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 28840, "BytesSent": 2727275.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133980, "ProcId": 0, "ImageSize": 30000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 12, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_2149965,ChtcWrapper10200.out,R2011b_INFO,AuditLog.10200,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133933, "ExitBySignal": false, "LastMatchTime": 1446133932, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 12, "DAGManJobId": 49583804, "MemoryUsage": 4, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 180, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 48, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/10200/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 48.0d, "LastJobLeaseRenewal": 1446133980, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "10200+10200", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_16@e022.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 9.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/10200/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133980, "StreamErr": false, "RecentBlockReadKbytes": 180, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583854, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 37, "CommittedSlotTime": 48.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=10200 --", "Environment": "", "LastPublicClaimId": "<128.104.58.32:31836>#1445317370#2779#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/10200", "QDate": 1446133826, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583856.0#1446133831", "StatsLifetimeStarter": 45, "JobStartDate": 1446133932, "SubmitEventNotes": "DAG Node: 11010+11010", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38257, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.104.58.32", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133932, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 5000, "RemoteWallClockTime": 47.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1206312, "EnteredCurrentStatus": 1446133979, "ResidentSetSize_RAW": 4812, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 15.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 4812, "BytesSent": 2727276.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133979, "ProcId": 0, "ImageSize": 5000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 3, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_2149966,ChtcWrapper11010.out,R2011b_INFO,AuditLog.11010,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133934, "ExitBySignal": false, "LastMatchTime": 1446133932, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 3, "DAGManJobId": 49583804, "MemoryUsage": 4, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 36, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 47, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/11010/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 47.0d, "LastJobLeaseRenewal": 1446133979, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "11010+11010", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_17@e022.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 11.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/11010/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133979, "StreamErr": false, "RecentBlockReadKbytes": 36, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583856, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 37, "CommittedSlotTime": 47.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=11010 --", "Environment": "", "LastPublicClaimId": "<128.104.58.32:31836>#1445317370#2817#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/11010", "QDate": 1446133831, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583894.0#1446133871", "StatsLifetimeStarter": 42, "JobStartDate": 1446133964, "SubmitEventNotes": "DAG Node: 00202+00202", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.58.28", "WantGlidein": true, "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133964, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 7500, "RemoteWallClockTime": 44.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 32, "DiskUsage_RAW": 1206795, "EnteredCurrentStatus": 1446134008, "ResidentSetSize_RAW": 5056, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5064, "BytesSent": 2727272.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446134008, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 6, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_517597,ChtcWrapper00202.out,R2011b_INFO,AuditLog.00202,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 32, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133965, "ExitBySignal": false, "LastMatchTime": 1446133964, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 6, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 156, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 44, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/00202/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 44.0d, "LastJobLeaseRenewal": 1446134008, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220267.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "00202+00202", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_23@e018.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 7.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/00202/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134008, "StreamErr": false, "RecentBlockReadKbytes": 156, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583894, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 32, "CommittedSlotTime": 44.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=00202 --", "Environment": "", "LastPublicClaimId": "<128.104.58.28:7648>#1445363387#2925#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/00202", "QDate": 1446133871, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583905.0#1446133888", "StatsLifetimeStarter": 76, "JobStartDate": 1446133963, "SubmitEventNotes": "DAG Node: 10012+10012", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38267, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.105.244.69", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133963, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 7500, "RemoteWallClockTime": 77.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1211433, "EnteredCurrentStatus": 1446134040, "ResidentSetSize_RAW": 5128, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5128, "BytesSent": 2727355.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134040, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 12, "SpooledOutputFiles": "R2011b_INFO,CODEBLOWUP,AuditLog.10012,SLIBS2.tar.gz,ChtcWrapper10012.out,CURLTIME_2575055,chtcinnerwrapper", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133964, "ExitBySignal": false, "LastMatchTime": 1446133963, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 12, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 160, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 77, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/10012/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 77.0d, "LastJobLeaseRenewal": 1446134040, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "10012+10012", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_2@e189.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 12.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/10012/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134040, "StreamErr": false, "RecentBlockReadKbytes": 160, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583905, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 67, "CommittedSlotTime": 77.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=10012 --", "Environment": "", "LastPublicClaimId": "<128.105.244.69:4177>#1444973293#3769#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/10012", "QDate": 1446133888, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583931.0#1446133916", "StatsLifetimeStarter": 49, "JobStartDate": 1446133964, "SubmitEventNotes": "DAG Node: 21020+21020", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38259, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.105.245.36", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133964, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 7500, "RemoteWallClockTime": 51.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1118707, "EnteredCurrentStatus": 1446134015, "ResidentSetSize_RAW": 5124, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 14.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5124, "BytesSent": 2727283.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134015, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 3, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_1010832,ChtcWrapper21020.out,R2011b_INFO,AuditLog.21020,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133965, "ExitBySignal": false, "LastMatchTime": 1446133964, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 3, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 12, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 51, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/21020/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 51.0d, "LastJobLeaseRenewal": 1446134015, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "21020+21020", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_10@e236.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 8.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/21020/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134015, "StreamErr": false, "RecentBlockReadKbytes": 12, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583931, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 40, "CommittedSlotTime": 51.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=21020 --", "Environment": "", "LastPublicClaimId": "<128.105.245.36:40852>#1445025971#4139#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/21020", "QDate": 1446133916, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583938.0#1446133922", "StatsLifetimeStarter": 52, "JobStartDate": 1446133963, "SubmitEventNotes": "DAG Node: 20111+20111", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38259, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.105.244.37", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133963, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 7500, "RemoteWallClockTime": 58.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 249656, "DiskUsage_RAW": 1205568, "EnteredCurrentStatus": 1446134021, "ResidentSetSize_RAW": 5056, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 11.0d, "BlockWrites": 506, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5056, "BytesSent": 2727274.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 506, "JobFinishedHookDone": 1446134021, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 16, "SpooledOutputFiles": "chtcinnerwrapper,SLIBS2.tar.gz,R2011b_INFO,AuditLog.20111,CURLTIME_1051736,ChtcWrapper20111.out,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 249656, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133964, "ExitBySignal": false, "LastMatchTime": 1446133963, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 16, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 164, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 58, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/20111/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 58.0d, "LastJobLeaseRenewal": 1446134021, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "20111+20111", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_10@e168.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 7.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/20111/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134021, "StreamErr": false, "RecentBlockReadKbytes": 164, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583938, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 43, "CommittedSlotTime": 58.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=20111 --", "Environment": "", "LastPublicClaimId": "<128.105.244.37:57713>#1445396629#2313#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/20111", "QDate": 1446133922, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
-{ "GlobalJobId": "submit-3.chtc.wisc.edu#49584018.0#1446134012", "StatsLifetimeStarter": 56, "JobStartDate": 1446134107, "SubmitEventNotes": "DAG Node: 11021+11021", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.39", "WantGlidein": true, "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446134107, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 7500, "RemoteWallClockTime": 58.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1139127, "EnteredCurrentStatus": 1446134165, "ResidentSetSize_RAW": 5124, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 ); ", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 14.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5124, "BytesSent": 2727270.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134165, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 14, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_137795,ChtcWrapper11021.out,R2011b_INFO,AuditLog.11021,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446134109, "ExitBySignal": false, "LastMatchTime": 1446134107, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 14, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 ); ", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 160, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 58, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/11021/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 58.0d, "LastJobLeaseRenewal": 1446134165, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "11021+11021", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e239.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 12.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/11021/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134165, "StreamErr": false, "RecentBlockReadKbytes": 160, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer ); ", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49584018, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 48, "CommittedSlotTime": 58.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=11021 --", "Environment": "", "LastPublicClaimId": "<128.105.245.39:54850>#1445038698#5043#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/11021", "QDate": 1446134012, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
\ No newline at end of file
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#48968872.0#1445354636", "StatsLifetimeStarter": 572059, "JobStartDate": 1445362267, "SubmitEventNotes": "DAG Node: fabp4-0002+fabp4-0002", "JobStatus": 4, "LeaveJobInQueue": false, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.104.119.175", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1445561276, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "ScheddBday": 1445383086, "RemoteWallClockTime": 769511.0d, "WantCheckpoint": false, "In": "/dev/null", "LastVacateTime": 1445546251, "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 32543, "EnteredCurrentStatus": 1446133322, "ResidentSetSize_RAW": 100432, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/ssericksen/dude-14-xdock/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 571737.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 690056, "BytesSent": 3113566.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133322, "ProcId": 0, "ImageSize": 750000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 12, "RecentBlockReads": 0, "SpooledOutputFiles": "ChtcWrapperfabp4-0002.out,AuditLog.fabp4-0002,poses.mol2,CURLTIME_4057178,harvest.log,time_elapsed.log,surf_scores.txt,CURLTIME_38803,count.log,fabp4-0002.out,CURLTIME_253463", "NumJobReconnects": 1, "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT_OR_EVICT", "JobCurrentStartExecutingDate": 1445561278, "ExitBySignal": false, "LastMatchTime": 1445561276, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 6, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 48940805, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 6, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 572046, "ExecutableSize_RAW": 6, "LastRejMatchReason": "no match found", "LastSuspensionTime": 0, "UserLog": "/home/ssericksen/dude-14-xdock/ChtcRun/dude14-surf-out-esr1/fabp4-0002/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 769511.0d, "LastJobLeaseRenewal": 1446133322, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 8.7351688E7d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "fabp4-0002+fabp4-0002", "PeriodicRelease": "error", "JobRunCount": 7, "LastRemotePool": "condor.biochem.wisc.edu:9618?sock=collector", "JobLastStartDate": 1445546257, "LastRemoteHost": "slot1@cluster-0008.biochem.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 0.0d, "TransferInput": "/home/ssericksen/dude-14-xdock/ChtcRun/dude14-surf-in/fabp4-0002/,/home/ssericksen/dude-14-xdock/ChtcRun/dude14-surf-in/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133322, "StreamErr": false, "is_resumable": true, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 7, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/ssericksen/dude-14-xdock/ChtcRun/dude14-surf-out-esr1/./mydag.dag.nodes.log", "Owner": "ssericksen", "Requirements": "undefined", "DiskUsage": 35000, "LastRejMatchTime": 1445375317, "JobLeaseDuration": 2400, "ClusterId": 48968872, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 572046.0d, "Args": "--type=Other --cmdtorun=surflex_run_DUDE_v1.8_esr1.sh --unique=fabp4-0002 --", "Environment": "", "LastPublicClaimId": "<128.104.119.175:9618>#1444067179#3317#...", "Iwd": "/home/ssericksen/dude-14-xdock/ChtcRun/dude14-surf-out-esr1/fabp4-0002", "QDate": 1445354636, "CurrentHosts": 0, "User": "ssericksen@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49573720.0#1445938922", "StatsLifetimeStarter": 190245, "JobStartDate": 1445943852, "SubmitEventNotes": "DAG Node: 180+180", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.72", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1445943852, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 190247.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134099, "ResidentSetSize_RAW": 123680, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 185236.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30766.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134099, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3753852,ChtcWrapper180.out,AuditLog.180,simu_3_180.txt,harvest.log,180.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1445943853, "ExitBySignal": false, "LastMatchTime": 1445943852, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49572657, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 190247, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally/Simulation_condor/model_3/180/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 190247.0d, "LastJobLeaseRenewal": 1446134099, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284367.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "180+180", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e272.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 1835.0d, "TransferInput": "/home/xguo23/finally/Simulation_condor/data/180/,/home/xguo23/finally/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134099, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49573720, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 190247.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=180 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.72:29075>#1444753997#6000#...", "Iwd": "/home/xguo23/finally/Simulation_condor/model_3/180", "QDate": 1445938922, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49581952.0#1446105329", "StatsLifetimeStarter": 27674, "JobStartDate": 1446106061, "SubmitEventNotes": "DAG Node: 40+40", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.86", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446106061, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 27676.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133737, "ResidentSetSize_RAW": 127252, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 27510.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30584.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133737, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_369560,ChtcWrapper40.out,AuditLog.40,simu_3_40.txt,harvest.log,40.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446106063, "ExitBySignal": false, "LastMatchTime": 1446106061, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 27676, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/40/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 27676.0d, "LastJobLeaseRenewal": 1446133737, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285053.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "40+40", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e286.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 105.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/40/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133737, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49581952, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 27676.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=40 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.86:32129>#1444759888#6329#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/40", "QDate": 1446105329, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49581985.0#1446105368", "StatsLifetimeStarter": 26354, "JobStartDate": 1446106289, "SubmitEventNotes": "DAG Node: 36+36", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.244.249", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446106289, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 26357.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132646, "ResidentSetSize_RAW": 127452, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 26239.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31898.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132646, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1151700,ChtcWrapper36.out,AuditLog.36,simu_3_36.txt,harvest.log,36.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446106291, "ExitBySignal": false, "LastMatchTime": 1446106289, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 26357, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/36/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 26357.0d, "LastJobLeaseRenewal": 1446132646, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285053.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "36+36", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e457.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 96.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/36/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132646, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49581985, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 26357.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=36 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.244.249:28476>#1444685646#10655#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/36", "QDate": 1446105368, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49581989.0#1446105374", "StatsLifetimeStarter": 27490, "JobStartDate": 1446106290, "SubmitEventNotes": "DAG Node: 82+82", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.233", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446106290, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 27491.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133781, "ResidentSetSize_RAW": 126932, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 27288.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30553.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133782, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_4096502,ChtcWrapper82.out,AuditLog.82,simu_3_82.txt,harvest.log,82.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446106291, "ExitBySignal": false, "LastMatchTime": 1446106290, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 27491, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/82/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 27491.0d, "LastJobLeaseRenewal": 1446133781, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285053.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "82+82", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e433.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 173.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/82/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133781, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49581989, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 27491.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=82 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.233:28601>#1443991451#13496#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/82", "QDate": 1446105374, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582049.0#1446105441", "StatsLifetimeStarter": 26296, "JobStartDate": 1446106482, "SubmitEventNotes": "DAG Node: 112+112", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.245", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446106482, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 26298.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132780, "ResidentSetSize_RAW": 126892, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 26097.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31904.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132780, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2601607,ChtcWrapper112.out,AuditLog.112,simu_3_112.txt,harvest.log,112.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446106484, "ExitBySignal": false, "LastMatchTime": 1446106482, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 26298, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/112/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 26298.0d, "LastJobLeaseRenewal": 1446132780, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "112+112", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e445.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 164.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/112/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132780, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582049, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 26298.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=112 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.245:48407>#1443991450#14631#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/112", "QDate": 1446105441, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582050.0#1446105441", "StatsLifetimeStarter": 27141, "JobStartDate": 1446106482, "SubmitEventNotes": "DAG Node: 301+301", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.172", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446106482, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 27143.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133625, "ResidentSetSize_RAW": 126464, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 26895.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31905.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133625, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2158419,ChtcWrapper301.out,AuditLog.301,simu_3_301.txt,harvest.log,301.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446106484, "ExitBySignal": false, "LastMatchTime": 1446106482, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 27143, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/301/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 27143.0d, "LastJobLeaseRenewal": 1446133625, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "301+301", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e372.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 201.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/301/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133625, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582050, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 27143.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=301 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.172:19856>#1444760019#9307#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/301", "QDate": 1446105441, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582065.0#1446105458", "StatsLifetimeStarter": 25606, "JobStartDate": 1446107042, "SubmitEventNotes": "DAG Node: 401+401", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.206", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107042, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 25607.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132649, "ResidentSetSize_RAW": 126608, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 25478.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30661.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132649, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1160521,ChtcWrapper401.out,AuditLog.401,simu_3_401.txt,harvest.log,401.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107043, "ExitBySignal": false, "LastMatchTime": 1446107042, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25607, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/401/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25607.0d, "LastJobLeaseRenewal": 1446132649, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "401+401", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e406.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 89.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/401/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132649, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582065, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25607.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=401 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.206:27946>#1443991437#15826#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/401", "QDate": 1446105458, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582094.0#1446105491", "StatsLifetimeStarter": 25168, "JobStartDate": 1446107489, "SubmitEventNotes": "DAG Node: 106+106", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.83", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107489, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 75000, "RemoteWallClockTime": 25169.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 4, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132658, "ResidentSetSize_RAW": 72016, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24949.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 119520, "BytesSent": 30486.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446132658, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 86, "SpooledOutputFiles": "CURLTIME_122139,ChtcWrapper106.out,AuditLog.106,simu_3_106.txt,harvest.log,106.out", "BlockWriteKbytes": 4, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107490, "ExitBySignal": false, "LastMatchTime": 1446107489, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 665, "DAGManJobId": 49581933, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 26620, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25169, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/106/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25169.0d, "LastJobLeaseRenewal": 1446132658, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "106+106", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c064.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 204.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/106/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132658, "StreamErr": false, "RecentBlockReadKbytes": 960, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582094, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25169.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=106 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.83:25899>#1445308581#1240#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/106", "QDate": 1446105491, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582098.0#1446105492", "StatsLifetimeStarter": 26020, "JobStartDate": 1446107489, "SubmitEventNotes": "DAG Node: 304+304", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.223", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107489, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 26022.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133511, "ResidentSetSize_RAW": 128776, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 25844.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31801.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133511, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3651606,ChtcWrapper304.out,AuditLog.304,simu_3_304.txt,harvest.log,304.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107491, "ExitBySignal": false, "LastMatchTime": 1446107489, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 26022, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/304/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 26022.0d, "LastJobLeaseRenewal": 1446133511, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "304+304", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e423.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 143.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/304/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133511, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582098, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 26022.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=304 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.223:13467>#1444760039#6376#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/304", "QDate": 1446105492, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582113.0#1446105509", "StatsLifetimeStarter": 26044, "JobStartDate": 1446107490, "SubmitEventNotes": "DAG Node: 206+206", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.120", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107490, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 26045.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133535, "ResidentSetSize_RAW": 126460, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 25939.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30596.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133535, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_522843,ChtcWrapper206.out,AuditLog.206,simu_3_206.txt,harvest.log,206.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107491, "ExitBySignal": false, "LastMatchTime": 1446107490, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 26045, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/206/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 26045.0d, "LastJobLeaseRenewal": 1446133535, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "206+206", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e320.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 87.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/206/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133535, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582113, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 26045.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=206 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.120:45185>#1443991409#14238#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/206", "QDate": 1446105509, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582119.0#1446105519", "StatsLifetimeStarter": 24928, "JobStartDate": 1446107490, "SubmitEventNotes": "DAG Node: 152+152", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.242", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107490, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24930.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132420, "ResidentSetSize_RAW": 128972, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24742.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30431.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132420, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_674,ChtcWrapper152.out,AuditLog.152,simu_3_152.txt,harvest.log,152.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107491, "ExitBySignal": false, "LastMatchTime": 1446107490, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24930, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/152/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24930.0d, "LastJobLeaseRenewal": 1446132420, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "152+152", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e442.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 156.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/152/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132420, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582119, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24930.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=152 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.242:38884>#1443991450#10374#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/152", "QDate": 1446105519, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582148.0#1446105547", "StatsLifetimeStarter": 26230, "JobStartDate": 1446107686, "SubmitEventNotes": "DAG Node: 162+162", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.170", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107686, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 26233.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133919, "ResidentSetSize_RAW": 126384, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 26088.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30612.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133919, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1114551,ChtcWrapper162.out,AuditLog.162,simu_3_162.txt,harvest.log,162.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107688, "ExitBySignal": false, "LastMatchTime": 1446107686, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 26233, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/162/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 26233.0d, "LastJobLeaseRenewal": 1446133919, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "162+162", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e370.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 96.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/162/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133919, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582148, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 26233.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=162 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.170:9482>#1443991414#13008#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/162", "QDate": 1446105547, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582154.0#1446105553", "StatsLifetimeStarter": 25874, "JobStartDate": 1446107686, "SubmitEventNotes": "DAG Node: 333+333", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.120", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446107686, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 25876.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133562, "ResidentSetSize_RAW": 125740, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 25692.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30542.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133562, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_523030,ChtcWrapper333.out,AuditLog.333,simu_3_333.txt,harvest.log,333.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446107688, "ExitBySignal": false, "LastMatchTime": 1446107686, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25876, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/333/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25876.0d, "LastJobLeaseRenewal": 1446133562, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "333+333", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e320.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 157.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/333/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133562, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582154, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25876.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=333 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.120:45185>#1443991409#14242#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/333", "QDate": 1446105553, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582177.0#1446105581", "StatsLifetimeStarter": 25025, "JobStartDate": 1446108665, "SubmitEventNotes": "DAG Node: 145+145", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.57", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108665, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 75000, "RemoteWallClockTime": 25026.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 4, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133691, "ResidentSetSize_RAW": 73308, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24770.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 120972, "BytesSent": 28290.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446133691, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 146, "SpooledOutputFiles": "CURLTIME_4179033,ChtcWrapper145.out,AuditLog.145,simu_3_145.txt,harvest.log,145.out", "BlockWriteKbytes": 4, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108666, "ExitBySignal": false, "LastMatchTime": 1446108665, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 796, "DAGManJobId": 49581933, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 28476, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25026, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/145/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25026.0d, "LastJobLeaseRenewal": 1446133691, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "145+145", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c038.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 217.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/145/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133691, "StreamErr": false, "RecentBlockReadKbytes": 1932, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582177, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25026.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=145 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.57:49793>#1445322694#1541#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/145", "QDate": 1446105581, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582178.0#1446105581", "StatsLifetimeStarter": 24871, "JobStartDate": 1446108666, "SubmitEventNotes": "DAG Node: 154+154", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.158", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108666, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24874.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133540, "ResidentSetSize_RAW": 125792, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24626.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30559.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133540, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1382128,ChtcWrapper154.out,AuditLog.154,simu_3_154.txt,harvest.log,154.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108668, "ExitBySignal": false, "LastMatchTime": 1446108666, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24874, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/154/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24874.0d, "LastJobLeaseRenewal": 1446133540, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "154+154", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e358.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 183.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/154/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133540, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582178, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24874.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=154 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.158:24962>#1444759998#9379#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/154", "QDate": 1446105581, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582181.0#1446105586", "StatsLifetimeStarter": 25146, "JobStartDate": 1446108665, "SubmitEventNotes": "DAG Node: 181+181", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.102", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108665, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 25148.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133813, "ResidentSetSize_RAW": 125368, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24957.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30557.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133813, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3925831,ChtcWrapper181.out,AuditLog.181,simu_3_181.txt,harvest.log,181.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108667, "ExitBySignal": false, "LastMatchTime": 1446108665, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25148, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/181/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25148.0d, "LastJobLeaseRenewal": 1446133813, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "181+181", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e302.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 148.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/181/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133813, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582181, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25148.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=181 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.102:26944>#1443991374#13401#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/181", "QDate": 1446105586, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582187.0#1446105592", "StatsLifetimeStarter": 25238, "JobStartDate": 1446108666, "SubmitEventNotes": "DAG Node: 343+343", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.141", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108666, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 25241.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133907, "ResidentSetSize_RAW": 127540, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 25080.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31798.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133907, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2577757,ChtcWrapper343.out,AuditLog.343,simu_3_343.txt,harvest.log,343.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108668, "ExitBySignal": false, "LastMatchTime": 1446108666, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25241, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/343/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25241.0d, "LastJobLeaseRenewal": 1446133907, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "343+343", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e341.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 127.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/343/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133907, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582187, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25241.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=343 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.141:7534>#1444673425#9467#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/343", "QDate": 1446105592, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582211.0#1446105612", "StatsLifetimeStarter": 23957, "JobStartDate": 1446108667, "SubmitEventNotes": "DAG Node: 5+5", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.105", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108667, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 23958.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132625, "ResidentSetSize_RAW": 125804, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 23804.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30545.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132625, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1839815,ChtcWrapper5.out,AuditLog.5,simu_3_5.txt,harvest.log,5.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108668, "ExitBySignal": false, "LastMatchTime": 1446108667, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 23958, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/5/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 23958.0d, "LastJobLeaseRenewal": 1446132625, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284627.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "5+5", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e305.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 120.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/5/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132625, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582211, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 23958.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=5 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.105:48578>#1445357425#5008#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/5", "QDate": 1446105612, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582222.0#1446105617", "StatsLifetimeStarter": 24898, "JobStartDate": 1446108828, "SubmitEventNotes": "DAG Node: 6+6", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.67", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108828, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24900.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133728, "ResidentSetSize_RAW": 126592, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24729.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30543.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133728, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1151628,ChtcWrapper6.out,AuditLog.6,simu_3_6.txt,harvest.log,6.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108830, "ExitBySignal": false, "LastMatchTime": 1446108828, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24900, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/6/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24900.0d, "LastJobLeaseRenewal": 1446133728, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284627.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "6+6", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e267.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 129.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/6/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133728, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582222, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24900.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=6 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.67:65111>#1444759823#5994#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/6", "QDate": 1446105617, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582230.0#1446105620", "StatsLifetimeStarter": 24122, "JobStartDate": 1446108827, "SubmitEventNotes": "DAG Node: 182+182", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.62", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108827, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24124.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132951, "ResidentSetSize_RAW": 125656, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 23953.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30489.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132951, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_511744,ChtcWrapper182.out,AuditLog.182,simu_3_182.txt,harvest.log,182.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108829, "ExitBySignal": false, "LastMatchTime": 1446108827, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24124, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/182/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24124.0d, "LastJobLeaseRenewal": 1446132951, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "182+182", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e262.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 126.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/182/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132951, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582230, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24124.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=182 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.62:1966>#1444680938#10248#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/182", "QDate": 1446105620, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582236.0#1446105622", "StatsLifetimeStarter": 25080, "JobStartDate": 1446108827, "SubmitEventNotes": "DAG Node: 10+10", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.245", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108827, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 25082.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133909, "ResidentSetSize_RAW": 126500, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24966.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30659.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133909, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2607549,ChtcWrapper10.out,AuditLog.10,simu_3_10.txt,harvest.log,10.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108829, "ExitBySignal": false, "LastMatchTime": 1446108827, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25082, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/10/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25082.0d, "LastJobLeaseRenewal": 1446133909, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "10+10", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e445.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 83.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/10/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133909, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582236, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25082.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=10 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.245:48407>#1443991450#14655#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/10", "QDate": 1446105622, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582256.0#1446105629", "StatsLifetimeStarter": 24797, "JobStartDate": 1446108828, "SubmitEventNotes": "DAG Node: 23+23", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.244.247", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108828, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 24798.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133626, "ResidentSetSize_RAW": 126760, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24631.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30534.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133627, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_908930,ChtcWrapper23.out,AuditLog.23,simu_3_23.txt,harvest.log,23.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108829, "ExitBySignal": false, "LastMatchTime": 1446108828, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24798, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/23/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24798.0d, "LastJobLeaseRenewal": 1446133626, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "23+23", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e455.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 132.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/23/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133626, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582256, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24798.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=23 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.244.247:44193>#1444685638#5758#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/23", "QDate": 1446105629, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582261.0#1446105631", "StatsLifetimeStarter": 25132, "JobStartDate": 1446108995, "SubmitEventNotes": "DAG Node: 407+407", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38210, "StartdPrincipal": "execute-side@matchsession/128.104.55.48", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446108995, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 100000, "RemoteWallClockTime": 25133.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134128, "ResidentSetSize_RAW": 76112, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24776.0d, "BlockWrites": 4, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 123648, "BytesSent": 30561.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134128, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 313, "SpooledOutputFiles": "harvest.log,ChtcWrapper407.out,AuditLog.407,CURLTIME_1861323,407.out,simu_3_407.txt", "BlockWriteKbytes": 16, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446108996, "ExitBySignal": false, "LastMatchTime": 1446108995, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 906, "DAGManJobId": 49581933, "MemoryUsage": 97, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 30280, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 25133, "ExecutableSize_RAW": 6, "LastRejMatchReason": "PREEMPTION_REQUIREMENTS == False ", "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/407/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 25133.0d, "LastJobLeaseRenewal": 1446134128, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "407+407", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c029.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 277.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/407/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134128, "StreamErr": false, "RecentBlockReadKbytes": 3976, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "LastRejMatchTime": 1446108994, "JobLeaseDuration": 2400, "ClusterId": 49582261, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 25133.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=407 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.48:26476>#1445344800#1604#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/407", "QDate": 1446105631, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582281.0#1446105639", "StatsLifetimeStarter": 23559, "JobStartDate": 1446109353, "SubmitEventNotes": "DAG Node: 24+24", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.149", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109353, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 23560.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132913, "ResidentSetSize_RAW": 127800, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 23403.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27904.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132913, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2096182,ChtcWrapper24.out,AuditLog.24,simu_3_24.txt,harvest.log,24.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446109354, "ExitBySignal": false, "LastMatchTime": 1446109353, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 23560, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/24/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 23560.0d, "LastJobLeaseRenewal": 1446132913, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "24+24", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e349.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 118.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/24/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132913, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582281, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 23560.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=24 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.149:6629>#1443991419#14390#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/24", "QDate": 1446105639, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582283.0#1446105640", "StatsLifetimeStarter": 24295, "JobStartDate": 1446109353, "SubmitEventNotes": "DAG Node: 33+33", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.75", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109353, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24297.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133650, "ResidentSetSize_RAW": 126564, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 24147.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31684.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133650, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3304508,ChtcWrapper33.out,AuditLog.33,simu_3_33.txt,harvest.log,33.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446109354, "ExitBySignal": false, "LastMatchTime": 1446109353, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24297, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/33/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24297.0d, "LastJobLeaseRenewal": 1446133650, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "33+33", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e275.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 135.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/33/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133650, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582283, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24297.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=33 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.75:36755>#1444759846#8529#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/33", "QDate": 1446105640, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582308.0#1446105649", "StatsLifetimeStarter": 23044, "JobStartDate": 1446109803, "SubmitEventNotes": "DAG Node: 25+25", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.190", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109803, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 23045.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132848, "ResidentSetSize_RAW": 126180, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22891.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30497.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132848, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_4129250,ChtcWrapper25.out,AuditLog.25,simu_3_25.txt,harvest.log,25.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446109804, "ExitBySignal": false, "LastMatchTime": 1446109803, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 23045, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/25/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 23045.0d, "LastJobLeaseRenewal": 1446132848, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "25+25", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e390.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 132.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/25/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132848, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582308, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 23045.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=25 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.190:40807>#1443991430#14737#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/25", "QDate": 1446105649, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582313.0#1446105651", "StatsLifetimeStarter": 24043, "JobStartDate": 1446109803, "SubmitEventNotes": "DAG Node: 61+61", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.92", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109803, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 24044.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133847, "ResidentSetSize_RAW": 128692, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 23894.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30500.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133847, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3652530,ChtcWrapper61.out,AuditLog.61,simu_3_61.txt,harvest.log,61.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446109804, "ExitBySignal": false, "LastMatchTime": 1446109803, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 24044, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/61/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 24044.0d, "LastJobLeaseRenewal": 1446133847, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "61+61", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e292.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 113.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/61/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133847, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582313, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 24044.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=61 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.92:44347>#1444759907#8412#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/61", "QDate": 1446105651, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582323.0#1446105655", "StatsLifetimeStarter": 22981, "JobStartDate": 1446109998, "SubmitEventNotes": "DAG Node: 61+61", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.83", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109998, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 75000, "RemoteWallClockTime": 22983.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 4, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132981, "ResidentSetSize_RAW": 72244, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22740.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 119748, "BytesSent": 30533.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446132981, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 71, "SpooledOutputFiles": "CURLTIME_127008,ChtcWrapper61.out,AuditLog.61,simu_3_61.txt,harvest.log,61.out", "BlockWriteKbytes": 4, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110000, "ExitBySignal": false, "LastMatchTime": 1446109998, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 808, "DAGManJobId": 49582206, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 37312, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22983, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/61/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22983.0d, "LastJobLeaseRenewal": 1446132981, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "61+61", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c064.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 197.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/61/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132981, "StreamErr": false, "RecentBlockReadKbytes": 848, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582323, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22983.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=61 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.83:25899>#1445308581#1248#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/61", "QDate": 1446105655, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582339.0#1446105660", "StatsLifetimeStarter": 22784, "JobStartDate": 1446109999, "SubmitEventNotes": "DAG Node: 35+35", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.177", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446109999, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22787.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132786, "ResidentSetSize_RAW": 125340, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22613.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30552.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132786, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3107604,ChtcWrapper35.out,AuditLog.35,simu_3_35.txt,harvest.log,35.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110001, "ExitBySignal": false, "LastMatchTime": 1446109999, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22787, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/35/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22787.0d, "LastJobLeaseRenewal": 1446132786, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "35+35", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e377.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 130.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/35/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132786, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582339, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22787.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=35 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.177:46087>#1443991411#13647#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/35", "QDate": 1446105660, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582357.0#1446105667", "StatsLifetimeStarter": 22635, "JobStartDate": 1446110000, "SubmitEventNotes": "DAG Node: 18+18", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.248", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110000, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22636.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132636, "ResidentSetSize_RAW": 127300, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22506.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27904.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132636, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2816308,ChtcWrapper18.out,AuditLog.18,simu_3_18.txt,harvest.log,18.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110001, "ExitBySignal": false, "LastMatchTime": 1446110000, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22636, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/18/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22636.0d, "LastJobLeaseRenewal": 1446132636, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "18+18", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e448.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 111.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/18/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132636, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582357, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22636.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=18 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.248:41700>#1443991446#11545#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/18", "QDate": 1446105667, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582358.0#1446105667", "StatsLifetimeStarter": 22588, "JobStartDate": 1446110000, "SubmitEventNotes": "DAG Node: 36+36", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.226", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110000, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22590.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132590, "ResidentSetSize_RAW": 125920, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22431.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30587.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132590, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_533867,ChtcWrapper36.out,AuditLog.36,simu_3_36.txt,harvest.log,36.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110002, "ExitBySignal": false, "LastMatchTime": 1446110000, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22590, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/36/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22590.0d, "LastJobLeaseRenewal": 1446132590, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "36+36", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e426.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 130.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/36/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132590, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582358, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22590.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=36 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.226:11484>#1443991456#11499#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/36", "QDate": 1446105667, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582391.0#1446105679", "StatsLifetimeStarter": 23041, "JobStartDate": 1446110203, "SubmitEventNotes": "DAG Node: 83+83", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.244.249", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110203, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 23043.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133246, "ResidentSetSize_RAW": 127748, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22845.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27908.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133246, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1162426,ChtcWrapper83.out,AuditLog.83,simu_3_83.txt,harvest.log,83.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110205, "ExitBySignal": false, "LastMatchTime": 1446110203, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 23043, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/83/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 23043.0d, "LastJobLeaseRenewal": 1446133246, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "83+83", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e457.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 142.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/83/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133246, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582391, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 23043.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=83 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.244.249:28476>#1444685646#10673#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/83", "QDate": 1446105679, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582404.0#1446105684", "StatsLifetimeStarter": 22108, "JobStartDate": 1446110586, "SubmitEventNotes": "DAG Node: 29+29", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.21", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110586, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 75000, "RemoteWallClockTime": 22109.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132695, "ResidentSetSize_RAW": 70692, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21939.0d, "BlockWrites": 2, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 118196, "BytesSent": 28013.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132695, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 3, "SpooledOutputFiles": "harvest.log,simu_3_29.txt,ChtcWrapper29.out,AuditLog.29,29.out,CURLTIME_3320245", "BlockWriteKbytes": 8, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110587, "ExitBySignal": false, "LastMatchTime": 1446110586, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 811, "DAGManJobId": 49582200, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 39868, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22109, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/29/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22109.0d, "LastJobLeaseRenewal": 1446132695, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "29+29", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c002.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 129.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/29/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132695, "StreamErr": false, "RecentBlockReadKbytes": 12, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582404, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22109.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=29 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.21:40483>#1445289732#1640#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/29", "QDate": 1446105684, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582416.0#1446105688", "StatsLifetimeStarter": 21892, "JobStartDate": 1446110790, "SubmitEventNotes": "DAG Node: 47+47", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.94", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110790, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 21894.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132684, "ResidentSetSize_RAW": 126208, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21739.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 29074.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132684, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_143217,ChtcWrapper47.out,AuditLog.47,simu_3_47.txt,harvest.log,47.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110792, "ExitBySignal": false, "LastMatchTime": 1446110790, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21894, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/47/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21894.0d, "LastJobLeaseRenewal": 1446132684, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "47+47", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e294.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 121.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/47/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132684, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582416, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21894.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=47 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.94:64588>#1444759915#9064#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/47", "QDate": 1446105688, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582421.0#1446105690", "StatsLifetimeStarter": 22909, "JobStartDate": 1446110790, "SubmitEventNotes": "DAG Node: 66+66", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.244.247", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110790, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22911.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133701, "ResidentSetSize_RAW": 126672, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22800.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30534.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133701, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_915408,ChtcWrapper66.out,AuditLog.66,simu_3_66.txt,harvest.log,66.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110792, "ExitBySignal": false, "LastMatchTime": 1446110790, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22911, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/66/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22911.0d, "LastJobLeaseRenewal": 1446133701, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "66+66", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e455.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 81.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/66/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133701, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582421, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22911.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=66 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.244.247:44193>#1444685638#5766#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/66", "QDate": 1446105690, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582432.0#1446105695", "StatsLifetimeStarter": 22249, "JobStartDate": 1446110791, "SubmitEventNotes": "DAG Node: 39+39", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.244.247", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110791, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22250.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133041, "ResidentSetSize_RAW": 125680, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22111.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27942.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133041, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_915422,ChtcWrapper39.out,AuditLog.39,simu_3_39.txt,harvest.log,39.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110792, "ExitBySignal": false, "LastMatchTime": 1446110791, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22250, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/39/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22250.0d, "LastJobLeaseRenewal": 1446133041, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "39+39", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e455.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 119.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/39/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133041, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582432, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22250.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=39 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.244.247:44193>#1444685638#5772#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/39", "QDate": 1446105695, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582440.0#1446105697", "StatsLifetimeStarter": 22526, "JobStartDate": 1446110791, "SubmitEventNotes": "DAG Node: 382+382", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.123", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446110791, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22528.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133319, "ResidentSetSize_RAW": 125040, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22357.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30506.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133319, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2233833,ChtcWrapper382.out,AuditLog.382,simu_3_382.txt,harvest.log,382.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446110793, "ExitBySignal": false, "LastMatchTime": 1446110791, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22528, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/382/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22528.0d, "LastJobLeaseRenewal": 1446133319, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "382+382", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e323.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 135.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/382/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133319, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582440, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22528.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=382 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.123:60803>#1444759965#6770#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/382", "QDate": 1446105697, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582460.0#1446105706", "StatsLifetimeStarter": 22393, "JobStartDate": 1446111074, "SubmitEventNotes": "DAG Node: 58+58", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.184", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111074, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22394.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133468, "ResidentSetSize_RAW": 127308, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22278.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30499.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133468, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2507136,ChtcWrapper58.out,AuditLog.58,simu_3_58.txt,harvest.log,58.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111075, "ExitBySignal": false, "LastMatchTime": 1446111074, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22394, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/58/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22394.0d, "LastJobLeaseRenewal": 1446133468, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "58+58", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e384.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 81.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/58/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133468, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582460, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22394.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=58 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.184:62907>#1443991428#13854#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/58", "QDate": 1446105706, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582462.0#1446105706", "StatsLifetimeStarter": 22519, "JobStartDate": 1446111074, "SubmitEventNotes": "DAG Node: 86+86", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.185", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111074, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22520.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133594, "ResidentSetSize_RAW": 125772, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22371.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30498.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133594, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3353135,ChtcWrapper86.out,AuditLog.86,simu_3_86.txt,harvest.log,86.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111075, "ExitBySignal": false, "LastMatchTime": 1446111074, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22520, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/86/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22520.0d, "LastJobLeaseRenewal": 1446133594, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "86+86", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e385.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 120.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/86/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133594, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582462, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22520.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=86 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.185:43838>#1443991427#12472#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/86", "QDate": 1446105706, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582463.0#1446105706", "StatsLifetimeStarter": 21403, "JobStartDate": 1446111073, "SubmitEventNotes": "DAG Node: 77+77", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.131", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111073, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 21404.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132477, "ResidentSetSize_RAW": 124948, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21243.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27912.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132477, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1137029,ChtcWrapper77.out,AuditLog.77,simu_3_77.txt,harvest.log,77.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111074, "ExitBySignal": false, "LastMatchTime": 1446111073, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21404, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/77/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21404.0d, "LastJobLeaseRenewal": 1446132477, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "77+77", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e331.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 134.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/77/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132477, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582463, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21404.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=77 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.131:14956>#1444819395#7764#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/77", "QDate": 1446105706, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582467.0#1446105708", "StatsLifetimeStarter": 22208, "JobStartDate": 1446111073, "SubmitEventNotes": "DAG Node: 275+275", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.87", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111073, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22209.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133282, "ResidentSetSize_RAW": 126040, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22110.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30485.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133282, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_626043,ChtcWrapper275.out,AuditLog.275,simu_3_275.txt,harvest.log,275.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111074, "ExitBySignal": false, "LastMatchTime": 1446111073, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22209, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/275/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22209.0d, "LastJobLeaseRenewal": 1446133282, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "275+275", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e287.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 78.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/275/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133282, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582467, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22209.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=275 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.87:2102>#1444759894#8469#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/275", "QDate": 1446105708, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582476.0#1446105711", "StatsLifetimeStarter": 22252, "JobStartDate": 1446111073, "SubmitEventNotes": "DAG Node: 96+96", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.127", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111073, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 22253.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133326, "ResidentSetSize_RAW": 127304, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22096.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28231.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133328, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2637948,ChtcWrapper96.out,AuditLog.96,simu_3_96.txt,harvest.log,96.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111074, "ExitBySignal": false, "LastMatchTime": 1446111073, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22253, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/96/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22253.0d, "LastJobLeaseRenewal": 1446133326, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "96+96", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e327.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 124.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/96/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133326, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582476, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22253.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=96 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.127:48134>#1443991405#8554#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/96", "QDate": 1446105711, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582487.0#1446105716", "StatsLifetimeStarter": 22573, "JobStartDate": 1446111074, "SubmitEventNotes": "DAG Node: 87+87", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.127", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111074, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22575.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133649, "ResidentSetSize_RAW": 128908, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22397.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27905.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133649, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2637965,ChtcWrapper87.out,AuditLog.87,simu_3_87.txt,harvest.log,87.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111076, "ExitBySignal": false, "LastMatchTime": 1446111074, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22575, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/87/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22575.0d, "LastJobLeaseRenewal": 1446133649, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "87+87", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e327.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 125.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/87/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133649, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582487, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22575.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=87 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.127:48134>#1443991405#8558#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/87", "QDate": 1446105716, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582489.0#1446105717", "StatsLifetimeStarter": 22699, "JobStartDate": 1446111074, "SubmitEventNotes": "DAG Node: 69+69", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.237", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111074, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22702.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133776, "ResidentSetSize_RAW": 126444, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22552.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30552.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133776, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1529941,ChtcWrapper69.out,AuditLog.69,simu_3_69.txt,harvest.log,69.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111076, "ExitBySignal": false, "LastMatchTime": 1446111074, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22702, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/69/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22702.0d, "LastJobLeaseRenewal": 1446133776, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "69+69", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e437.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 122.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/69/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133776, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582489, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22702.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=69 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.237:1373>#1444673410#8302#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/69", "QDate": 1446105717, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582491.0#1446105717", "StatsLifetimeStarter": 22279, "JobStartDate": 1446111075, "SubmitEventNotes": "DAG Node: 88+88", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.184", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111075, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 22280.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133355, "ResidentSetSize_RAW": 126084, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22117.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27905.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133355, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2507151,ChtcWrapper88.out,AuditLog.88,simu_3_88.txt,harvest.log,88.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111076, "ExitBySignal": false, "LastMatchTime": 1446111075, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22280, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/88/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22280.0d, "LastJobLeaseRenewal": 1446133355, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284612.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "88+88", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e384.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 144.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/88/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133355, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582491, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22280.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=88 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.184:62907>#1443991428#13858#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/88", "QDate": 1446105717, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582494.0#1446105717", "StatsLifetimeStarter": 21401, "JobStartDate": 1446111075, "SubmitEventNotes": "DAG Node: 89+89", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.97", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111075, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 21403.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132478, "ResidentSetSize_RAW": 126072, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21263.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30508.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132478, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1266453,ChtcWrapper89.out,AuditLog.89,simu_3_89.txt,harvest.log,89.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111077, "ExitBySignal": false, "LastMatchTime": 1446111075, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21403, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/89/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21403.0d, "LastJobLeaseRenewal": 1446132478, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284628.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "89+89", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e297.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 114.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/89/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132478, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582494, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21403.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=89 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.97:50007>#1444685419#10092#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/89", "QDate": 1446105717, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582510.0#1446105723", "StatsLifetimeStarter": 21254, "JobStartDate": 1446111276, "SubmitEventNotes": "DAG Node: 210+210", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.66", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111276, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 100000, "RemoteWallClockTime": 21256.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 4, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132532, "ResidentSetSize_RAW": 75336, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21046.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 122840, "BytesSent": 30503.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446132532, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 540, "SpooledOutputFiles": "CURLTIME_1268212,ChtcWrapper210.out,AuditLog.210,simu_3_210.txt,harvest.log,210.out", "BlockWriteKbytes": 4, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111278, "ExitBySignal": false, "LastMatchTime": 1446111276, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 1407, "DAGManJobId": 49582200, "MemoryUsage": 97, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 51368, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21256, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/210/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21256.0d, "LastJobLeaseRenewal": 1446132532, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "210+210", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c047.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 169.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/210/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132532, "StreamErr": false, "RecentBlockReadKbytes": 8940, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582510, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21256.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=210 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.66:54632>#1445311857#1452#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/210", "QDate": 1446105723, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582511.0#1446105723", "StatsLifetimeStarter": 22242, "JobStartDate": 1446111276, "SubmitEventNotes": "DAG Node: 201+201", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 13, "StartdPrincipal": "execute-side@matchsession/128.104.55.68", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111276, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 75000, "RemoteWallClockTime": 22243.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 4, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133519, "ResidentSetSize_RAW": 72052, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22043.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 117304, "BytesSent": 30483.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446133520, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 460, "SpooledOutputFiles": "CURLTIME_935737,ChtcWrapper201.out,AuditLog.201,simu_3_201.txt,harvest.log,201.out", "BlockWriteKbytes": 4, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111277, "ExitBySignal": false, "LastMatchTime": 1446111276, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 1906, "DAGManJobId": 49582200, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 81056, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22243, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/201/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22243.0d, "LastJobLeaseRenewal": 1446133519, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "201+201", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c049.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 133.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/201/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133519, "StreamErr": false, "RecentBlockReadKbytes": 4868, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582511, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22243.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=201 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.68:4958>#1445345121#1580#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/201", "QDate": 1446105723, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582525.0#1446105728", "StatsLifetimeStarter": 22429, "JobStartDate": 1446111452, "SubmitEventNotes": "DAG Node: 300+300", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.126", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111452, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22430.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133882, "ResidentSetSize_RAW": 126740, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22300.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133883, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1700927,ChtcWrapper300.out,AuditLog.300,simu_3_300.txt,harvest.log,300.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111453, "ExitBySignal": false, "LastMatchTime": 1446111452, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22430, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/300/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22430.0d, "LastJobLeaseRenewal": 1446133882, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "300+300", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e326.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 112.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/300/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133882, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582525, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22430.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=300 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.126:40098>#1444759970#7928#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/300", "QDate": 1446105728, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582533.0#1446105734", "StatsLifetimeStarter": 22519, "JobStartDate": 1446111647, "SubmitEventNotes": "DAG Node: 211+211", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.61", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111647, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 22520.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134167, "ResidentSetSize_RAW": 126608, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 22353.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30603.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134167, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_818403,ChtcWrapper211.out,AuditLog.211,simu_3_211.txt,harvest.log,211.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111648, "ExitBySignal": false, "LastMatchTime": 1446111647, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22520, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/211/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22520.0d, "LastJobLeaseRenewal": 1446134167, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "211+211", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e261.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 137.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/211/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134167, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582533, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22520.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=211 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.61:49736>#1444759807#6759#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/211", "QDate": 1446105734, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582539.0#1446105734", "StatsLifetimeStarter": 21532, "JobStartDate": 1446111811, "SubmitEventNotes": "DAG Node: 121+121", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.102", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111811, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 21534.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133345, "ResidentSetSize_RAW": 125956, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21392.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30557.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133345, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3934729,ChtcWrapper121.out,AuditLog.121,simu_3_121.txt,harvest.log,121.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111812, "ExitBySignal": false, "LastMatchTime": 1446111811, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21534, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/121/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21534.0d, "LastJobLeaseRenewal": 1446133345, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "121+121", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e302.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 122.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/121/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133345, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582539, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21534.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=121 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.102:26944>#1443991374#13421#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/121", "QDate": 1446105734, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582540.0#1446105734", "StatsLifetimeStarter": 22050, "JobStartDate": 1446111810, "SubmitEventNotes": "DAG Node: 220+220", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.126", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111810, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 22052.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133862, "ResidentSetSize_RAW": 126940, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21897.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28344.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133862, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1702594,ChtcWrapper220.out,AuditLog.220,simu_3_220.txt,harvest.log,220.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111812, "ExitBySignal": false, "LastMatchTime": 1446111810, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 22052, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/220/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 22052.0d, "LastJobLeaseRenewal": 1446133862, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "220+220", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e326.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 127.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/220/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133862, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582540, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 22052.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=220 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.126:40098>#1444759970#7932#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/220", "QDate": 1446105734, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582553.0#1446105740", "StatsLifetimeStarter": 21632, "JobStartDate": 1446111993, "SubmitEventNotes": "DAG Node: 121+121", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.141", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446111993, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 21635.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133628, "ResidentSetSize_RAW": 126224, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21477.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30505.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133628, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2585208,ChtcWrapper121.out,AuditLog.121,simu_3_121.txt,harvest.log,121.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446111995, "ExitBySignal": false, "LastMatchTime": 1446111993, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21635, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/121/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21635.0d, "LastJobLeaseRenewal": 1446133628, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "121+121", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e341.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 134.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/121/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133628, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582553, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21635.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=121 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.141:7534>#1444673425#9485#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/121", "QDate": 1446105740, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582557.0#1446105741", "StatsLifetimeStarter": 21953, "JobStartDate": 1446112222, "SubmitEventNotes": "DAG Node: 159+159", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.152", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446112222, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 21954.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134176, "ResidentSetSize_RAW": 125604, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 21791.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30561.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134177, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2696692,ChtcWrapper159.out,AuditLog.159,simu_3_159.txt,harvest.log,159.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446112223, "ExitBySignal": false, "LastMatchTime": 1446112222, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 147, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 21954, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/159/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 21954.0d, "LastJobLeaseRenewal": 1446134176, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "159+159", "PeriodicRelease": false, "JobRunCount": 1, "LastRemoteHost": "slot1@e352.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 137.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/159/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134176, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": false, "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582557, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 21954.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=159 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.152:39021>#1444772294#9281#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/159", "QDate": 1446105741, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582585.0#1446105751", "StatsLifetimeStarter": 20690, "JobStartDate": 1446112544, "SubmitEventNotes": "DAG Node: 311+311", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.244", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446112544, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 20692.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133236, "ResidentSetSize_RAW": 126832, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 20568.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27894.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133236, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_363317,ChtcWrapper311.out,AuditLog.311,simu_3_311.txt,harvest.log,311.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446112547, "ExitBySignal": false, "LastMatchTime": 1446112544, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 20692, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/311/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 20692.0d, "LastJobLeaseRenewal": 1446133236, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "311+311", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e444.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 87.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/311/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133236, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582585, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 20692.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=311 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.244:1411>#1443991446#13076#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/311", "QDate": 1446105751, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582591.0#1446105753", "StatsLifetimeStarter": 19824, "JobStartDate": 1446112695, "SubmitEventNotes": "DAG Node: 438+438", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.211", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446112695, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 19825.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132520, "ResidentSetSize_RAW": 125924, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 19694.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 29426.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132520, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1907890,ChtcWrapper438.out,AuditLog.438,simu_3_438.txt,harvest.log,438.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446112696, "ExitBySignal": false, "LastMatchTime": 1446112695, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 19825, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/438/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 19825.0d, "LastJobLeaseRenewal": 1446132520, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "438+438", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e411.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 107.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/438/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132520, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582591, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 19825.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=438 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.211:65149>#1443991444#12482#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/438", "QDate": 1446105753, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582595.0#1446105757", "StatsLifetimeStarter": 20366, "JobStartDate": 1446112695, "SubmitEventNotes": "DAG Node: 104+104", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.193", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446112695, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 20367.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133062, "ResidentSetSize_RAW": 125640, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 20221.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31674.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133062, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_173815,ChtcWrapper104.out,AuditLog.104,simu_3_104.txt,harvest.log,104.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446112696, "ExitBySignal": false, "LastMatchTime": 1446112695, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 20367, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/104/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 20367.0d, "LastJobLeaseRenewal": 1446133062, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "104+104", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e393.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 126.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/104/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133062, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582595, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 20367.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=104 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.193:65434>#1443991433#12882#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/104", "QDate": 1446105757, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582601.0#1446105757", "StatsLifetimeStarter": 19359, "JobStartDate": 1446113038, "SubmitEventNotes": "DAG Node: 231+231", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.197", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446113038, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 19360.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132398, "ResidentSetSize_RAW": 125720, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 19226.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30507.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132398, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3355874,ChtcWrapper231.out,AuditLog.231,simu_3_231.txt,harvest.log,231.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446113039, "ExitBySignal": false, "LastMatchTime": 1446113038, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 19360, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/231/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 19360.0d, "LastJobLeaseRenewal": 1446132398, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "231+231", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e397.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 111.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/231/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132398, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582601, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 19360.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=231 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.197:37993>#1443991431#13795#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/231", "QDate": 1446105757, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582610.0#1446105762", "StatsLifetimeStarter": 20371, "JobStartDate": 1446113038, "SubmitEventNotes": "DAG Node: 213+213", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.219", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446113038, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 20372.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133410, "ResidentSetSize_RAW": 124944, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 20214.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30507.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133410, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2647255,ChtcWrapper213.out,AuditLog.213,simu_3_213.txt,harvest.log,213.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446113039, "ExitBySignal": false, "LastMatchTime": 1446113038, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 20372, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/213/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 20372.0d, "LastJobLeaseRenewal": 1446133410, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "213+213", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e419.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 132.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/213/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133410, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582610, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 20372.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=213 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.219:51004>#1443991439#14297#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/213", "QDate": 1446105762, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582641.0#1446105773", "StatsLifetimeStarter": 18408, "JobStartDate": 1446114160, "SubmitEventNotes": "DAG Node: 105+105", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.166", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446114160, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 18410.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132570, "ResidentSetSize_RAW": 124020, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18271.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28241.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132570, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1464388,ChtcWrapper105.out,AuditLog.105,simu_3_105.txt,harvest.log,105.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446114162, "ExitBySignal": false, "LastMatchTime": 1446114160, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18410, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/105/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18410.0d, "LastJobLeaseRenewal": 1446132570, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "105+105", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e366.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 118.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/105/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132570, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582641, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18410.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=105 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.166:20019>#1444831317#8851#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/105", "QDate": 1446105773, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582659.0#1446105779", "StatsLifetimeStarter": 19336, "JobStartDate": 1446114724, "SubmitEventNotes": "DAG Node: 232+232", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.55.48", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446114724, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 75000, "RemoteWallClockTime": 19338.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216668, "EnteredCurrentStatus": 1446134062, "ResidentSetSize_RAW": 71268, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 19081.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 118772, "BytesSent": 27911.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134062, "ProcId": 0, "ImageSize": 125000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 3, "SpooledOutputFiles": "harvest.log,232.out,ChtcWrapper232.out,AuditLog.232,CURLTIME_1864147,simu_3_232.txt", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446114726, "ExitBySignal": false, "LastMatchTime": 1446114724, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 615, "DAGManJobId": 49582200, "MemoryUsage": 73, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 26436, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 19338, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/232/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 19338.0d, "LastJobLeaseRenewal": 1446134062, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "232+232", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@c029.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 179.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/232/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134062, "StreamErr": false, "RecentBlockReadKbytes": 12, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582659, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 19338.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=232 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.55.48:26476>#1445344800#1612#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/232", "QDate": 1446105779, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582699.0#1446105797", "StatsLifetimeStarter": 18984, "JobStartDate": 1446114861, "SubmitEventNotes": "DAG Node: 313+313", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.157", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446114861, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 18985.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133846, "ResidentSetSize_RAW": 126756, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18847.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27896.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133846, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_930697,ChtcWrapper313.out,AuditLog.313,simu_3_313.txt,harvest.log,313.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446114862, "ExitBySignal": false, "LastMatchTime": 1446114861, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18985, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/313/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18985.0d, "LastJobLeaseRenewal": 1446133846, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "313+313", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e357.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 107.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/313/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133846, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582699, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18985.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=313 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.157:33109>#1444685526#8861#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/313", "QDate": 1446105797, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582722.0#1446105803", "StatsLifetimeStarter": 18066, "JobStartDate": 1446115114, "SubmitEventNotes": "DAG Node: 298+298", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.197", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115114, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 18067.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133181, "ResidentSetSize_RAW": 127108, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/finally_2/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17941.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 31693.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133181, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3362144,ChtcWrapper298.out,AuditLog.298,simu_3_298.txt,harvest.log,298.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115115, "ExitBySignal": false, "LastMatchTime": 1446115114, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49581933, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18067, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/finally_2/Simulation_condor/model_3/298/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18067.0d, "LastJobLeaseRenewal": 1446133181, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 285054.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "298+298", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e397.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 110.0d, "TransferInput": "/home/xguo23/finally_2/Simulation_condor/data/298/,/home/xguo23/finally_2/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133181, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/finally_2/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582722, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18067.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=298 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.197:37993>#1443991431#13812#...", "Iwd": "/home/xguo23/finally_2/Simulation_condor/model_3/298", "QDate": 1446105803, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582724.0#1446105803", "StatsLifetimeStarter": 18902, "JobStartDate": 1446115114, "SubmitEventNotes": "DAG Node: 260+260", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.164", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115114, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 18903.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134017, "ResidentSetSize_RAW": 124924, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18782.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30507.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134017, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2890029,ChtcWrapper260.out,AuditLog.260,simu_3_260.txt,harvest.log,260.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115115, "ExitBySignal": false, "LastMatchTime": 1446115114, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18903, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/260/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18903.0d, "LastJobLeaseRenewal": 1446134017, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "260+260", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e364.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 109.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/260/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134017, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582724, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18903.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=260 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.164:7769>#1444760010#7999#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/260", "QDate": 1446105803, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582760.0#1446105819", "StatsLifetimeStarter": 18410, "JobStartDate": 1446115399, "SubmitEventNotes": "DAG Node: 170+170", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.113", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115399, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 18411.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133810, "ResidentSetSize_RAW": 125400, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18266.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28239.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133810, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2614862,ChtcWrapper170.out,AuditLog.170,simu_3_170.txt,harvest.log,170.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115400, "ExitBySignal": false, "LastMatchTime": 1446115399, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18411, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/170/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18411.0d, "LastJobLeaseRenewal": 1446133810, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "170+170", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e313.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 104.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/170/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133810, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582760, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18411.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=170 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.113:56191>#1443991385#10335#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/170", "QDate": 1446105819, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582771.0#1446105825", "StatsLifetimeStarter": 17065, "JobStartDate": 1446115399, "SubmitEventNotes": "DAG Node: 350+350", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.197", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115399, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 17066.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132465, "ResidentSetSize_RAW": 126136, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16934.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132465, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3362491,ChtcWrapper350.out,AuditLog.350,simu_3_350.txt,harvest.log,350.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115400, "ExitBySignal": false, "LastMatchTime": 1446115399, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17066, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/350/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17066.0d, "LastJobLeaseRenewal": 1446132465, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "350+350", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e397.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 103.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/350/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132465, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582771, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17066.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=350 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.197:37993>#1443991431#13821#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/350", "QDate": 1446105825, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582782.0#1446105831", "StatsLifetimeStarter": 18187, "JobStartDate": 1446115778, "SubmitEventNotes": "DAG Node: 440+440", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.158", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115778, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 18189.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133967, "ResidentSetSize_RAW": 125632, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18054.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27914.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133967, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1401618,ChtcWrapper440.out,AuditLog.440,simu_3_440.txt,harvest.log,440.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115779, "ExitBySignal": false, "LastMatchTime": 1446115778, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18189, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/440/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18189.0d, "LastJobLeaseRenewal": 1446133967, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "440+440", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e358.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 105.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/440/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133967, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582782, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18189.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=440 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.158:24962>#1444759998#9425#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/440", "QDate": 1446105831, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582783.0#1446105831", "StatsLifetimeStarter": 18022, "JobStartDate": 1446115778, "SubmitEventNotes": "DAG Node: 270+270", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.242", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115778, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 18023.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133801, "ResidentSetSize_RAW": 127404, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17875.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27877.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133801, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_24492,ChtcWrapper270.out,AuditLog.270,simu_3_270.txt,harvest.log,270.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115779, "ExitBySignal": false, "LastMatchTime": 1446115778, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18023, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/270/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18023.0d, "LastJobLeaseRenewal": 1446133801, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "270+270", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e442.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 115.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/270/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133801, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582783, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18023.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=270 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.242:38884>#1443991450#10410#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/270", "QDate": 1446105831, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582786.0#1446105835", "StatsLifetimeStarter": 18247, "JobStartDate": 1446115778, "SubmitEventNotes": "DAG Node: 3+3", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.107", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115778, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 18248.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134026, "ResidentSetSize_RAW": 125940, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 18118.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27896.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134026, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3140097,ChtcWrapper3.out,AuditLog.3,simu_3_3.txt,harvest.log,3.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115779, "ExitBySignal": false, "LastMatchTime": 1446115778, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 18248, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/3/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 18248.0d, "LastJobLeaseRenewal": 1446134026, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284717.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "3+3", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e307.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 108.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/3/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134026, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582786, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 18248.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=3 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.107:63744>#1444685448#11070#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/3", "QDate": 1446105835, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582794.0#1446105836", "StatsLifetimeStarter": 17555, "JobStartDate": 1446115778, "SubmitEventNotes": "DAG Node: 252+252", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.174", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115778, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 17557.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133335, "ResidentSetSize_RAW": 126656, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17422.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27913.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133335, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3591632,ChtcWrapper252.out,AuditLog.252,simu_3_252.txt,harvest.log,252.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115780, "ExitBySignal": false, "LastMatchTime": 1446115778, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17557, "ExecutableSize_RAW": 6, "LastRejMatchReason": "no match found", "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/252/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17557.0d, "LastJobLeaseRenewal": 1446133335, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "252+252", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e374.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 108.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/252/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133335, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "LastRejMatchTime": 1446115777, "JobLeaseDuration": 2400, "ClusterId": 49582794, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17557.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=252 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.174:7981>#1444760024#8714#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/252", "QDate": 1446105836, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582796.0#1446105836", "StatsLifetimeStarter": 17251, "JobStartDate": 1446115778, "SubmitEventNotes": "DAG Node: 243+243", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.225", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115778, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 17253.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133031, "ResidentSetSize_RAW": 127320, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17159.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27913.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133031, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2889473,ChtcWrapper243.out,AuditLog.243,simu_3_243.txt,harvest.log,243.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115780, "ExitBySignal": false, "LastMatchTime": 1446115778, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17253, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/243/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17253.0d, "LastJobLeaseRenewal": 1446133031, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "243+243", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e425.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 62.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/243/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133031, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582796, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17253.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=243 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.225:15992>#1444673418#9764#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/243", "QDate": 1446105836, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582827.0#1446105847", "StatsLifetimeStarter": 16585, "JobStartDate": 1446115924, "SubmitEventNotes": "DAG Node: 162+162", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.161", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115924, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 16587.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132511, "ResidentSetSize_RAW": 126964, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16462.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30506.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132511, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2195199,ChtcWrapper162.out,AuditLog.162,simu_3_162.txt,harvest.log,162.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115926, "ExitBySignal": false, "LastMatchTime": 1446115924, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16587, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/162/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16587.0d, "LastJobLeaseRenewal": 1446132511, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "162+162", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e361.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 94.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/162/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132511, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582827, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16587.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=162 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.161:7475>#1443991415#14144#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/162", "QDate": 1446105847, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582831.0#1446105852", "StatsLifetimeStarter": 17603, "JobStartDate": 1446115924, "SubmitEventNotes": "DAG Node: 31+31", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.65", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446115924, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 17605.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133529, "ResidentSetSize_RAW": 124912, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17471.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27905.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133529, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3813186,ChtcWrapper31.out,AuditLog.31,simu_3_31.txt,harvest.log,31.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446115925, "ExitBySignal": false, "LastMatchTime": 1446115924, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17604, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/31/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17605.0d, "LastJobLeaseRenewal": 1446133528, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284718.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "31+31", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e265.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 101.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/31/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133529, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582831, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17604.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=31 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.65:22193>#1444759815#9517#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/31", "QDate": 1446105852, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582862.0#1446105863", "StatsLifetimeStarter": 16713, "JobStartDate": 1446116497, "SubmitEventNotes": "DAG Node: 51+51", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.151", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116497, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 16714.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133211, "ResidentSetSize_RAW": 124628, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16576.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30499.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133211, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1045787,ChtcWrapper51.out,AuditLog.51,simu_3_51.txt,harvest.log,51.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116498, "ExitBySignal": false, "LastMatchTime": 1446116497, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16714, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/51/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16714.0d, "LastJobLeaseRenewal": 1446133211, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284718.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "51+51", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e351.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 103.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/51/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133211, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582862, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16714.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=51 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.151:14279>#1445444483#5155#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/51", "QDate": 1446105863, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582867.0#1446105863", "StatsLifetimeStarter": 16609, "JobStartDate": 1446116497, "SubmitEventNotes": "DAG Node: 423+423", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.75", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116497, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 16610.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133107, "ResidentSetSize_RAW": 127232, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16440.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28235.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133107, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3323529,ChtcWrapper423.out,AuditLog.423,simu_3_423.txt,harvest.log,423.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116498, "ExitBySignal": false, "LastMatchTime": 1446116497, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16610, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/423/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16610.0d, "LastJobLeaseRenewal": 1446133107, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "423+423", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e275.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 139.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/423/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133107, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582867, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16610.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=423 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.75:36755>#1444759846#8549#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/423", "QDate": 1446105863, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582873.0#1446105864", "StatsLifetimeStarter": 17505, "JobStartDate": 1446116497, "SubmitEventNotes": "DAG Node: 280+280", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.244", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116497, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 17506.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134003, "ResidentSetSize_RAW": 126200, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17357.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27894.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134003, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_373303,ChtcWrapper280.out,AuditLog.280,simu_3_280.txt,harvest.log,280.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116498, "ExitBySignal": false, "LastMatchTime": 1446116497, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17506, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/280/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17506.0d, "LastJobLeaseRenewal": 1446134003, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "280+280", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e444.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 112.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/280/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134003, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582873, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17506.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=280 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.244:1411>#1443991446#13109#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/280", "QDate": 1446105864, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582874.0#1446105868", "StatsLifetimeStarter": 15928, "JobStartDate": 1446116497, "SubmitEventNotes": "DAG Node: 70+70", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.194", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116497, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 15929.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132426, "ResidentSetSize_RAW": 126992, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 15804.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27908.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132426, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3390859,ChtcWrapper70.out,AuditLog.70,simu_3_70.txt,harvest.log,70.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116498, "ExitBySignal": false, "LastMatchTime": 1446116497, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 15929, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/70/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 15929.0d, "LastJobLeaseRenewal": 1446132426, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284718.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "70+70", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e394.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 100.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/70/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132426, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582874, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 15929.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=70 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.194:52833>#1443991432#16216#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/70", "QDate": 1446105868, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582878.0#1446105869", "StatsLifetimeStarter": 17433, "JobStartDate": 1446116497, "SubmitEventNotes": "DAG Node: 43+43", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.244", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116497, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 17435.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133932, "ResidentSetSize_RAW": 124328, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 17306.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30479.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133932, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_373315,ChtcWrapper43.out,AuditLog.43,simu_3_43.txt,harvest.log,43.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116499, "ExitBySignal": false, "LastMatchTime": 1446116497, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 17435, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/43/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 17435.0d, "LastJobLeaseRenewal": 1446133932, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284718.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "43+43", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e444.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 106.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/43/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133932, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582878, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 17435.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=43 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.244:1411>#1443991446#13114#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/43", "QDate": 1446105869, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582883.0#1446105869", "StatsLifetimeStarter": 16052, "JobStartDate": 1446116623, "SubmitEventNotes": "DAG Node: 450+450", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.188", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116623, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 16053.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132676, "ResidentSetSize_RAW": 125676, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 15939.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30507.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132676, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2688575,ChtcWrapper450.out,AuditLog.450,simu_3_450.txt,harvest.log,450.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116624, "ExitBySignal": false, "LastMatchTime": 1446116623, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16053, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/450/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16053.0d, "LastJobLeaseRenewal": 1446132676, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "450+450", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e388.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 86.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/450/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132676, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582883, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16053.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=450 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.188:28065>#1443991430#12995#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/450", "QDate": 1446105869, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582897.0#1446105875", "StatsLifetimeStarter": 16343, "JobStartDate": 1446116750, "SubmitEventNotes": "DAG Node: 226+226", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.235", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116750, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 16344.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133094, "ResidentSetSize_RAW": 124920, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16201.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133094, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3306268,ChtcWrapper226.out,AuditLog.226,simu_3_226.txt,harvest.log,226.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116751, "ExitBySignal": false, "LastMatchTime": 1446116750, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16344, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/226/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16344.0d, "LastJobLeaseRenewal": 1446133094, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "226+226", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e435.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 119.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/226/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133094, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582897, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16344.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=226 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.235:26914>#1443991459#10913#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/226", "QDate": 1446105875, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49582902.0#1446105875", "StatsLifetimeStarter": 16445, "JobStartDate": 1446116750, "SubmitEventNotes": "DAG Node: 136+136", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.194", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446116750, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 16446.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133196, "ResidentSetSize_RAW": 126576, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 16315.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30507.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133196, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3391975,ChtcWrapper136.out,AuditLog.136,simu_3_136.txt,harvest.log,136.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446116751, "ExitBySignal": false, "LastMatchTime": 1446116750, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 16446, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/136/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 16446.0d, "LastJobLeaseRenewal": 1446133196, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "136+136", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e394.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 106.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/136/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133196, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49582902, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 16446.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=136 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.194:52833>#1443991432#16220#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/136", "QDate": 1446105875, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583239.0#1446106003", "StatsLifetimeStarter": 13050, "JobStartDate": 1446121053, "SubmitEventNotes": "DAG Node: 409+409", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 24, "StartdPrincipal": "execute-side@matchsession/128.105.245.242", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121053, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 150000, "RemoteWallClockTime": 13051.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134104, "ResidentSetSize_RAW": 127216, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12934.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27873.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134104, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_37424,ChtcWrapper409.out,AuditLog.409,simu_3_409.txt,harvest.log,409.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121054, "ExitBySignal": false, "LastMatchTime": 1446121053, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 13051, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/409/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 13051.0d, "LastJobLeaseRenewal": 1446134104, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "409+409", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e442.chtc.WISC.EDU", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 93.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/409/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134104, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583239, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 13051.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=409 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.242:38884>#1443991450#10456#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/409", "QDate": 1446106003, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583254.0#1446106008", "StatsLifetimeStarter": 12361, "JobStartDate": 1446121052, "SubmitEventNotes": "DAG Node: 275+275", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.163", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121052, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 12363.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133415, "ResidentSetSize_RAW": 126732, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12249.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30506.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133415, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2948896,ChtcWrapper275.out,AuditLog.275,simu_3_275.txt,harvest.log,275.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121054, "ExitBySignal": false, "LastMatchTime": 1446121052, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 12363, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/275/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 12363.0d, "LastJobLeaseRenewal": 1446133415, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "275+275", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e363.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 95.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/275/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133415, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583254, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 12363.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=275 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.163:21972>#1443991420#10986#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/275", "QDate": 1446106008, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583271.0#1446106014", "StatsLifetimeStarter": 11405, "JobStartDate": 1446121053, "SubmitEventNotes": "DAG Node: 149+149", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.101.145", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121053, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 11407.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132460, "ResidentSetSize_RAW": 124492, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 11060.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132460, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3185190,ChtcWrapper149.out,AuditLog.149,simu_3_149.txt,harvest.log,149.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121055, "ExitBySignal": false, "LastMatchTime": 1446121053, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 11407, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/149/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 11407.0d, "LastJobLeaseRenewal": 1446132460, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "149+149", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e145.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 70.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/149/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132460, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583271, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 11407.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=149 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.101.145:57668>#1444053387#12271#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/149", "QDate": 1446106014, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583278.0#1446106014", "StatsLifetimeStarter": 12654, "JobStartDate": 1446121054, "SubmitEventNotes": "DAG Node: 239+239", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.140", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121054, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 12656.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133710, "ResidentSetSize_RAW": 124780, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12549.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 28220.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133710, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_548136,ChtcWrapper239.out,AuditLog.239,simu_3_239.txt,harvest.log,239.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121056, "ExitBySignal": false, "LastMatchTime": 1446121054, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 12656, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/239/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 12656.0d, "LastJobLeaseRenewal": 1446133710, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "239+239", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e340.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 88.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/239/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133710, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583278, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 12656.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=239 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.140:58412>#1444681013#9585#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/239", "QDate": 1446106014, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583285.0#1446106020", "StatsLifetimeStarter": 12499, "JobStartDate": 1446121054, "SubmitEventNotes": "DAG Node: 194+194", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.73", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121054, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 12501.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133555, "ResidentSetSize_RAW": 125892, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12398.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133555, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2796351,ChtcWrapper194.out,AuditLog.194,simu_3_194.txt,harvest.log,194.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121056, "ExitBySignal": false, "LastMatchTime": 1446121054, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 12501, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/194/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 12501.0d, "LastJobLeaseRenewal": 1446133555, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "194+194", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e273.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 87.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/194/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133555, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583285, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 12501.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=194 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.73:33900>#1444759838#9136#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/194", "QDate": 1446106019, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583286.0#1446106020", "StatsLifetimeStarter": 11906, "JobStartDate": 1446121054, "SubmitEventNotes": "DAG Node: 284+284", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.101.145", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121054, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 11908.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446132962, "ResidentSetSize_RAW": 122612, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 11624.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446132962, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_3185196,ChtcWrapper284.out,AuditLog.284,simu_3_284.txt,harvest.log,284.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121056, "ExitBySignal": false, "LastMatchTime": 1446121054, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 11908, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/284/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 11908.0d, "LastJobLeaseRenewal": 1446132962, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "284+284", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e145.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 73.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/284/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446132962, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583286, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 11908.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=284 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.101.145:57668>#1444053387#12274#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/284", "QDate": 1446106020, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583295.0#1446106024", "StatsLifetimeStarter": 12273, "JobStartDate": 1446121055, "SubmitEventNotes": "DAG Node: 421+421", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.73", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121055, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 12274.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133329, "ResidentSetSize_RAW": 127332, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.46/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12186.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 27915.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133329, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_2796365,ChtcWrapper421.out,AuditLog.421,simu_3_421.txt,harvest.log,421.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121056, "ExitBySignal": false, "LastMatchTime": 1446121055, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582778, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 12274, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/421/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 12274.0d, "LastJobLeaseRenewal": 1446133329, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284719.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "421+421", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e273.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 70.0d, "TransferInput": "/home/xguo23/model_3_1.46/Simulation_condor/data/421/,/home/xguo23/model_3_1.46/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133329, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583295, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 12274.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=421 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.73:33900>#1444759838#9139#...", "Iwd": "/home/xguo23/model_3_1.46/Simulation_condor/model_3/421", "QDate": 1446106024, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583316.0#1446106031", "StatsLifetimeStarter": 12602, "JobStartDate": 1446121412, "SubmitEventNotes": "DAG Node: 419+419", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.140", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121412, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 150000, "RemoteWallClockTime": 12604.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446134016, "ResidentSetSize_RAW": 125420, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.47/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12491.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30485.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134016, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_549258,ChtcWrapper419.out,AuditLog.419,simu_3_419.txt,harvest.log,419.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121413, "ExitBySignal": false, "LastMatchTime": 1446121412, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582200, "MemoryUsage": 146, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 12604, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/419/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 12604.0d, "LastJobLeaseRenewal": 1446134016, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284629.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "419+419", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e340.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 86.0d, "TransferInput": "/home/xguo23/model_3_1.47/Simulation_condor/data/419/,/home/xguo23/model_3_1.47/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134016, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583316, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 12604.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=419 -- 3", "Environment": "", "LastPublicClaimId": "<128.105.245.140:58412>#1444681013#9588#...", "Iwd": "/home/xguo23/model_3_1.47/Simulation_condor/model_3/419", "QDate": 1446106031, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583329.0#1446106036", "StatsLifetimeStarter": 11940, "JobStartDate": 1446121413, "SubmitEventNotes": "DAG Node: 392+392", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.101.129", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446121413, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 125000, "RemoteWallClockTime": 11942.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1216669, "EnteredCurrentStatus": 1446133355, "ResidentSetSize_RAW": 119932, "RequestDisk": 1000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/xguo23/model_3_1.48/Simulation_condor/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 11714.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 811948, "BytesSent": 30504.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133355, "ProcId": 0, "ImageSize": 1000000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 0, "RecentBlockReads": 0, "SpooledOutputFiles": "CURLTIME_1549103,ChtcWrapper392.out,AuditLog.392,simu_3_392.txt,harvest.log,392.out", "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446121414, "ExitBySignal": false, "LastMatchTime": 1446121413, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 0, "DAGManJobId": 49582206, "MemoryUsage": 122, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 0, "JobNotification": 0, "BlockReadKbytes": 0, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 11942, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/392/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 11942.0d, "LastJobLeaseRenewal": 1446133355, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 284613.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "392+392", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e129.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 81.0d, "TransferInput": "/home/xguo23/model_3_1.48/Simulation_condor/data/392/,/home/xguo23/model_3_1.48/Simulation_condor/data/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133355, "StreamErr": false, "RecentBlockReadKbytes": 0, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/./mydag.dag.nodes.log", "Owner": "xguo23", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583329, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 1200, "CommittedSlotTime": 11942.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=simu_condor --unique=392 -- 3", "Environment": "", "LastPublicClaimId": "<128.104.101.129:24642>#1444053399#13743#...", "Iwd": "/home/xguo23/model_3_1.48/Simulation_condor/model_3/392", "QDate": 1446106036, "CurrentHosts": 0, "User": "xguo23@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583811.0#1446133780", "StatsLifetimeStarter": 39, "JobStartDate": 1446133930, "SubmitEventNotes": "DAG Node: 01100+01100", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38254, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.105.245.6", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133930, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 7500, "RemoteWallClockTime": 40.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1206320, "EnteredCurrentStatus": 1446133970, "ResidentSetSize_RAW": 5044, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 10.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5048, "BytesSent": 2727275.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133971, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 8, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_2893592,ChtcWrapper01100.out,R2011b_INFO,AuditLog.01100,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133931, "ExitBySignal": false, "LastMatchTime": 1446133930, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 8, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 108, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 40, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/01100/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 40.0d, "LastJobLeaseRenewal": 1446133970, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "01100+01100", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e206.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 6.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/01100/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133970, "StreamErr": false, "RecentBlockReadKbytes": 108, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583811, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 30, "CommittedSlotTime": 40.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=01100 --", "Environment": "", "LastPublicClaimId": "<128.105.245.6:9783>#1444977535#2490#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/01100", "QDate": 1446133780, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583854.0#1446133826", "StatsLifetimeStarter": 46, "JobStartDate": 1446133932, "SubmitEventNotes": "DAG Node: 10200+10200", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38256, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.104.58.32", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133932, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 5000, "RemoteWallClockTime": 48.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1206292, "EnteredCurrentStatus": 1446133980, "ResidentSetSize_RAW": 4816, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 15.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 28840, "BytesSent": 2727275.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133980, "ProcId": 0, "ImageSize": 30000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 12, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_2149965,ChtcWrapper10200.out,R2011b_INFO,AuditLog.10200,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133933, "ExitBySignal": false, "LastMatchTime": 1446133932, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 12, "DAGManJobId": 49583804, "MemoryUsage": 4, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 180, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 48, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/10200/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 48.0d, "LastJobLeaseRenewal": 1446133980, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "10200+10200", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_16@e022.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 9.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/10200/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133980, "StreamErr": false, "RecentBlockReadKbytes": 180, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583854, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 37, "CommittedSlotTime": 48.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=10200 --", "Environment": "", "LastPublicClaimId": "<128.104.58.32:31836>#1445317370#2779#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/10200", "QDate": 1446133826, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583856.0#1446133831", "StatsLifetimeStarter": 45, "JobStartDate": 1446133932, "SubmitEventNotes": "DAG Node: 11010+11010", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38257, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.104.58.32", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133932, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 5000, "RemoteWallClockTime": 47.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1206312, "EnteredCurrentStatus": 1446133979, "ResidentSetSize_RAW": 4812, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 15.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 4812, "BytesSent": 2727276.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446133979, "ProcId": 0, "ImageSize": 5000, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 3, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_2149966,ChtcWrapper11010.out,R2011b_INFO,AuditLog.11010,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133934, "ExitBySignal": false, "LastMatchTime": 1446133932, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 3, "DAGManJobId": 49583804, "MemoryUsage": 4, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 36, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 47, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/11010/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 47.0d, "LastJobLeaseRenewal": 1446133979, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "11010+11010", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_17@e022.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 11.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/11010/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446133979, "StreamErr": false, "RecentBlockReadKbytes": 36, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583856, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 37, "CommittedSlotTime": 47.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=11010 --", "Environment": "", "LastPublicClaimId": "<128.104.58.32:31836>#1445317370#2817#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/11010", "QDate": 1446133831, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583894.0#1446133871", "StatsLifetimeStarter": 42, "JobStartDate": 1446133964, "SubmitEventNotes": "DAG Node: 00202+00202", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.104.58.28", "WantGlidein": true, "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133964, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 7500, "RemoteWallClockTime": 44.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 32, "DiskUsage_RAW": 1206795, "EnteredCurrentStatus": 1446134008, "ResidentSetSize_RAW": 5056, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12.0d, "BlockWrites": 1, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5064, "BytesSent": 2727272.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 1, "JobFinishedHookDone": 1446134008, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 6, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_517597,ChtcWrapper00202.out,R2011b_INFO,AuditLog.00202,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 32, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133965, "ExitBySignal": false, "LastMatchTime": 1446133964, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 6, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 156, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 44, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/00202/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 44.0d, "LastJobLeaseRenewal": 1446134008, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220267.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "00202+00202", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_23@e018.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 7.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/00202/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134008, "StreamErr": false, "RecentBlockReadKbytes": 156, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583894, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 32, "CommittedSlotTime": 44.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=00202 --", "Environment": "", "LastPublicClaimId": "<128.104.58.28:7648>#1445363387#2925#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/00202", "QDate": 1446133871, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583905.0#1446133888", "StatsLifetimeStarter": 76, "JobStartDate": 1446133963, "SubmitEventNotes": "DAG Node: 10012+10012", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38267, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.105.244.69", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133963, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 7500, "RemoteWallClockTime": 77.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1211433, "EnteredCurrentStatus": 1446134040, "ResidentSetSize_RAW": 5128, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 12.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5128, "BytesSent": 2727355.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134040, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 12, "SpooledOutputFiles": "R2011b_INFO,CODEBLOWUP,AuditLog.10012,SLIBS2.tar.gz,ChtcWrapper10012.out,CURLTIME_2575055,chtcinnerwrapper", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133964, "ExitBySignal": false, "LastMatchTime": 1446133963, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 12, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 160, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 77, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/10012/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 77.0d, "LastJobLeaseRenewal": 1446134040, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "10012+10012", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_2@e189.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 12.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/10012/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134040, "StreamErr": false, "RecentBlockReadKbytes": 160, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583905, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 67, "CommittedSlotTime": 77.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=10012 --", "Environment": "", "LastPublicClaimId": "<128.105.244.69:4177>#1444973293#3769#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/10012", "QDate": 1446133888, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583931.0#1446133916", "StatsLifetimeStarter": 49, "JobStartDate": 1446133964, "SubmitEventNotes": "DAG Node: 21020+21020", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38259, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.105.245.36", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133964, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 7500, "RemoteWallClockTime": 51.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1118707, "EnteredCurrentStatus": 1446134015, "ResidentSetSize_RAW": 5124, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 14.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5124, "BytesSent": 2727283.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134015, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 3, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_1010832,ChtcWrapper21020.out,R2011b_INFO,AuditLog.21020,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133965, "ExitBySignal": false, "LastMatchTime": 1446133964, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 3, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 12, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 51, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/21020/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 51.0d, "LastJobLeaseRenewal": 1446134015, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "21020+21020", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_10@e236.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 8.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/21020/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134015, "StreamErr": false, "RecentBlockReadKbytes": 12, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583931, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 40, "CommittedSlotTime": 51.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=21020 --", "Environment": "", "LastPublicClaimId": "<128.105.245.36:40852>#1445025971#4139#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/21020", "QDate": 1446133916, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49583938.0#1446133922", "StatsLifetimeStarter": 52, "JobStartDate": 1446133963, "SubmitEventNotes": "DAG Node: 20111+20111", "JobStatus": 4, "LeaveJobInQueue": false, "AutoClusterId": 38259, "WantGlidein": true, "StartdPrincipal": "execute-side@matchsession/128.105.244.37", "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446133963, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "ExitStatus": 0, "Rank": 0.0d, "ResidentSetSize": 7500, "RemoteWallClockTime": 58.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 249656, "DiskUsage_RAW": 1205568, "EnteredCurrentStatus": 1446134021, "ResidentSetSize_RAW": 5056, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 11.0d, "BlockWrites": 506, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5056, "BytesSent": 2727274.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "AutoClusterAttrs": "JobUniverse,LastCheckpointPlatform,NumCkpts,ClientMachine,_condor_RequestCpus,_condor_RequestDisk,_condor_RequestGPUs,_condor_RequestMemory,RequestCpus,RequestDisk,RequestGPUs,RequestMemory,BIOCHEM,MachineLastMatchTime,ConcurrencyLimits,NiceUser,Rank,Requirements,ImageSize,MemoryRequirements,User,RemoteGroup,SubmitterGroup,SubmitterUserPrio,Group,WIDsTheme,InteractiveJob,Is_Resumable,WantFlocking,WantGlidein,Scheduler,Owner,JobStart,MemoryUsage,IsExpressQueueJob,DiskUsage,HEP_VO,IsDesktop,OSG_VO,x509userproxysubject,PassedTest,IsLocalCMSJob,IsLocalCMSSlot,IsSAMSlot,IsSAMJob,MaxDiskTempC,IsDedicated,estimated_run_hours,IsCHTCSubmit,RequiresCVMFS,DynamicSlot,PartitionableSlot,Slot1_ExpectedMachineGracefulDrainingCompletion,Slot1_JobStarts,Slot1_SelfMonitorAge,Slot2_ExpectedMachineGracefulDrainingCompletion,Slot2_JobStarts,Slot2_SelfMonitorAge,Slot3_ExpectedMachineGracefulDrainingCompletion,Slot3_JobStarts,Slot3_SelfMonitorAge,Slot4_ExpectedMachineGracefulDrainingCompletion,Slot4_JobStarts,Slot4_SelfMonitorAge,Slot5_ExpectedMachineGracefulDrainingCompletion,Slot5_JobStarts,Slot5_SelfMonitorAge,Slot6_ExpectedMachineGracefulDrainingCompletion,Slot6_JobStarts,Slot6_SelfMonitorAge,Slot7_ExpectedMachineGracefulDrainingCompletion,Slot7_JobStarts,Slot7_SelfMonitorAge,Slot8_ExpectedMachineGracefulDrainingCompletion,Slot8_JobStarts,Slot8_SelfMonitorAge,Slot1_TotalTimeClaimedBusy,Slot1_TotalTimeUnclaimedIdle,Slot2_TotalTimeClaimedBusy,Slot2_TotalTimeUnclaimedIdle,Slot3_TotalTimeClaimedBusy,Slot3_TotalTimeUnclaimedIdle,Slot4_TotalTimeClaimedBusy,Slot4_TotalTimeUnclaimedIdle,Slot5_TotalTimeClaimedBusy,Slot5_TotalTimeUnclaimedIdle,Slot6_TotalTimeClaimedBusy,Slot6_TotalTimeUnclaimedIdle,Slot7_TotalTimeClaimedBusy,Slot7_TotalTimeUnclaimedIdle,Slot8_TotalTimeClaimedBusy,Slot8_TotalTimeUnclaimedIdle,Slot10_ExpectedMachineGracefulDrainingCompletion,Slot10_JobStarts,Slot10_SelfMonitorAge,Slot11_ExpectedMachineGracefulDrainingCompletion,Slot11_JobStarts,Slot11_SelfMonitorAge,Slot12_ExpectedMachineGracefulDrainingCompletion,Slot12_JobStarts,Slot12_SelfMonitorAge,Slot9_ExpectedMachineGracefulDrainingCompletion,Slot9_JobStarts,Slot9_SelfMonitorAge,Slot12_TotalTimeClaimedBusy,Slot10_TotalTimeClaimedBusy,Slot10_TotalTimeUnclaimedIdle,Slot11_TotalTimeClaimedBusy,Slot11_TotalTimeUnclaimedIdle,Slot12_TotalTimeUnclaimedIdle,Slot9_TotalTimeClaimedBusy,Slot9_TotalTimeUnclaimedIdle,Slot13_ExpectedMachineGracefulDrainingCompletion,Slot13_JobStarts,Slot13_SelfMonitorAge,Slot14_ExpectedMachineGracefulDrainingCompletion,Slot14_JobStarts,Slot14_SelfMonitorAge,Slot15_ExpectedMachineGracefulDrainingCompletion,Slot15_JobStarts,Slot15_SelfMonitorAge,Slot16_ExpectedMachineGracefulDrainingCompletion,Slot16_JobStarts,Slot16_SelfMonitorAge,IsResumable,WHEN_TO_TRANSFER_OUTPUT,_condor_Requestadmin_mutex_1,_condor_Requestadmin_mutex_2,_condor_Requestadmin_mutex_3,_condor_Requestmachine_token,Requestadmin_mutex_1,Requestadmin_mutex_2,Requestadmin_mutex_3,Requestmachine_token,nyehle,IsBuildJob,IsMatlabBuildJob,TotalJobRunTime,NodeOnline,Slot13_TotalTimeClaimedBusy,Slot13_TotalTimeUnclaimedIdle,Slot14_TotalTimeClaimedBusy,Slot14_TotalTimeUnclaimedIdle,Slot15_TotalTimeClaimedBusy,Slot15_TotalTimeUnclaimedIdle,Slot16_TotalTimeClaimedBusy,Slot16_TotalTimeUnclaimedIdle,TmpIsFull,trResumable,RequiresCMSFrontier,Slot17_ExpectedMachineGracefulDrainingCompletion,Slot17_JobStarts,Slot17_SelfMonitorAge,Slot17_TotalTimeClaimedBusy,Slot17_TotalTimeUnclaimedIdle,Slot18_ExpectedMachineGracefulDrainingCompletion,Slot18_JobStarts,Slot18_SelfMonitorAge,Slot18_TotalTimeClaimedBusy,Slot18_TotalTimeUnclaimedIdle,Slot19_ExpectedMachineGracefulDrainingCompletion,Slot19_JobStarts,Slot19_SelfMonitorAge,Slot19_TotalTimeClaimedBusy,Slot19_TotalTimeUnclaimedIdle,Slot20_ExpectedMachineGracefulDrainingCompletion,Slot20_JobStarts,Slot20_SelfMonitorAge,Slot20_TotalTimeClaimedBusy,Slot20_TotalTimeUnclaimedIdle,Slot21_ExpectedMachineGracefulDrainingCompletion,Slot21_JobStarts,Slot21_SelfMonitorAge,Slot21_TotalTimeClaimedBusy,Slot21_TotalTimeUnclaimedIdle,Slot22_ExpectedMachineGracefulDrainingCompletion,Slot22_JobStarts,Slot22_SelfMonitorAge,Slot22_TotalTimeClaimedBusy,Slot22_TotalTimeUnclaimedIdle,Slot23_ExpectedMachineGracefulDrainingCompletion,Slot23_JobStarts,Slot23_SelfMonitorAge,Slot23_TotalTimeClaimedBusy,Slot23_TotalTimeUnclaimedIdle,Slot24_ExpectedMachineGracefulDrainingCompletion,Slot24_JobStarts,Slot24_SelfMonitorAge,Slot24_TotalTimeClaimedBusy,Slot24_TotalTimeUnclaimedIdle,Slot25_ExpectedMachineGracefulDrainingCompletion,Slot25_JobStarts,Slot25_SelfMonitorAge,Slot25_TotalTimeClaimedBusy,Slot25_TotalTimeUnclaimedIdle,Slot26_ExpectedMachineGracefulDrainingCompletion,Slot26_JobStarts,Slot26_SelfMonitorAge,Slot26_TotalTimeClaimedBusy,Slot26_TotalTimeUnclaimedIdle,Slot27_ExpectedMachineGracefulDrainingCompletion,Slot27_JobStarts,Slot27_SelfMonitorAge,Slot27_TotalTimeClaimedBusy,Slot27_TotalTimeUnclaimedIdle,Slot28_ExpectedMachineGracefulDrainingCompletion,Slot28_JobStarts,Slot28_SelfMonitorAge,Slot28_TotalTimeClaimedBusy,Slot28_TotalTimeUnclaimedIdle,Slot29_ExpectedMachineGracefulDrainingCompletion,Slot29_JobStarts,Slot29_SelfMonitorAge,Slot29_TotalTimeClaimedBusy,Slot29_TotalTimeUnclaimedIdle,Slot30_ExpectedMachineGracefulDrainingCompletion,Slot30_JobStarts,Slot30_SelfMonitorAge,Slot30_TotalTimeClaimedBusy,Slot30_TotalTimeUnclaimedIdle,Slot31_ExpectedMachineGracefulDrainingCompletion,Slot31_JobStarts,Slot31_SelfMonitorAge,Slot31_TotalTimeClaimedBusy,Slot31_TotalTimeUnclaimedIdle,Slot32_ExpectedMachineGracefulDrainingCompletion,Slot32_JobStarts,Slot32_SelfMonitorAge,Slot32_TotalTimeClaimedBusy,Slot32_TotalTimeUnclaimedIdle,ResidentSetSize", "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 506, "JobFinishedHookDone": 1446134021, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 16, "SpooledOutputFiles": "chtcinnerwrapper,SLIBS2.tar.gz,R2011b_INFO,AuditLog.20111,CURLTIME_1051736,ChtcWrapper20111.out,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 249656, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446133964, "ExitBySignal": false, "LastMatchTime": 1446133963, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 16, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 164, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 58, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/20111/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 58.0d, "LastJobLeaseRenewal": 1446134021, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "20111+20111", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1_10@e168.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 7.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/20111/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134021, "StreamErr": false, "RecentBlockReadKbytes": 164, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49583938, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 43, "CommittedSlotTime": 58.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=20111 --", "Environment": "", "LastPublicClaimId": "<128.105.244.37:57713>#1445396629#2313#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/20111", "QDate": 1446133922, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
+{ "GlobalJobId": "submit-3.chtc.wisc.edu#49584018.0#1446134012", "StatsLifetimeStarter": 56, "JobStartDate": 1446134107, "SubmitEventNotes": "DAG Node: 11021+11021", "JobStatus": 4, "LeaveJobInQueue": false, "StartdPrincipal": "execute-side@matchsession/128.105.245.39", "WantGlidein": true, "WantRHEL6": true, "OnExitRemove": true, "JobCurrentStartDate": 1446134107, "CoreSize": 0, "MATCH_EXP_JOBGLIDEIN_ResourceName": "wisc.edu", "Rank": 0.0d, "ExitStatus": 0, "ResidentSetSize": 7500, "RemoteWallClockTime": 58.0d, "WantCheckpoint": false, "In": "/dev/null", "MaxHosts": 1, "RootDir": "/", "NumRestarts": 0, "RecentBlockWriteKbytes": 0, "DiskUsage_RAW": 1139127, "EnteredCurrentStatus": 1446134165, "ResidentSetSize_RAW": 5124, "RequestDisk": 4000000, "MyType": "Job", "PeriodicRemove": false, "Cmd": "/home/dentler/ChtcRun/chtcjobwrapper", "CondorVersion": "$CondorVersion: 8.5.0 Sep 16 2015 BuildID: 341710 $", "ShouldTransferFiles": "YES", "MemoryUsageExpr": "expr=( ( ResidentSetSize + 1023 ) / 1024 );\n", "TargetType": "Machine", "MinHosts": 1, "NumCkpts_RAW": 0, "RequestCpus": 1, "RemoteUserCpu": 14.0d, "BlockWrites": 0, "NiceUser": false, "Out": "process.out", "ImageSize_RAW": 5124, "BytesSent": 2727270.0d, "CumulativeSuspensionTime": 0, "TransferIn": false, "NumCkpts": 0, "Err": "process.err", "RecentBlockWrites": 0, "JobFinishedHookDone": 1446134165, "ProcId": 0, "ImageSize": 7500, "JobUniverse": 5, "EncryptExecuteDirectory": false, "TransferInputSizeMB": 1, "RecentBlockReads": 14, "SpooledOutputFiles": "chtcinnerwrapper,CURLTIME_137795,ChtcWrapper11021.out,R2011b_INFO,AuditLog.11021,SLIBS2.tar.gz,CODEBLOWUP", "WantFlocking": true, "BlockWriteKbytes": 0, "WhenToTransferOutput": "ON_EXIT", "JobCurrentStartExecutingDate": 1446134109, "ExitBySignal": false, "LastMatchTime": 1446134107, "OnExitHold": false, "OrigMaxHosts": 1, "RequestMemory": 1000, "NumJobStarts": 1, "TerminationPending": true, "TotalSuspensions": 0, "BlockReads": 14, "DAGManJobId": 49583804, "MemoryUsage": 7, "PeriodicReleaseExpr": "expr=( JobStatus == 5 ) && ( ( CurrentTime - EnteredCurrentStatus ) > 1800 ) && ( JobRunCount < 5 ) && ( HoldReasonCode != 6 ) && ( HoldReasonCode != 14 ) && ( HoldReasonCode != 22 );\n", "ExitCode": 5, "JobNotification": 0, "BlockReadKbytes": 160, "NumJobMatches": 1, "LocalUserCpu": 0.0d, "LastJobStatus": 2, "BufferBlockSize": 32768, "CommittedTime": 58, "ExecutableSize_RAW": 6, "LastSuspensionTime": 0, "Matlab": "R2011b", "UserLog": "/home/dentler/ChtcRun/project_auction/results_fix2/11021/process.log", "DAGManNodesMask": "0,1,2,4,5,7,9,10,11,12,13,16,17,24,27", "CumulativeSlotTime": 58.0d, "LastJobLeaseRenewal": 1446134165, "MachineAttrSlotWeight0": 1, "NumSystemHolds": 0, "BytesRecvd": 1220270.0d, "CondorPlatform": "$CondorPlatform: X86_64-RedHat_6.6 $", "JOBGLIDEIN_ResourceName": "$$([IfThenElse(IsUndefined(TARGET.GLIDEIN_ResourceName), IfThenElse(IsUndefined(TARGET.GLIDEIN_Site), \"wisc.edu\", TARGET.GLIDEIN_Site), TARGET.GLIDEIN_ResourceName)])", "DAGNodeName": "11021+11021", "PeriodicRelease": "error", "JobRunCount": 1, "LastRemoteHost": "slot1@e239.chtc.wisc.edu", "JobPrio": 0, "LocalSysCpu": 0.0d, "ExecutableSize": 7, "RemoteSysCpu": 12.0d, "TransferInput": "/home/dentler/ChtcRun/project_auction/11021/,/home/dentler/ChtcRun/project_auction/shared/", "PeriodicHold": false, "WantRemoteIO": true, "CommittedSuspensionTime": 0, "DAGParentNodeNames": "", "CompletionDate": 1446134165, "StreamErr": false, "RecentBlockReadKbytes": 160, "WantRemoteSyscalls": false, "RequirementsExpr": "expr=( ( OpSysMajorVer is 6 ) ) && ( MY.JobUniverse == 12 || MY.JobUniverse == 7 || MY.WantFlocking || MY.WantGlidein || TARGET.PoolName == \"CHTC\" || TARGET.COLLECTOR_HOST_STRING == \"infopool.cs.wisc.edu\" ) && ( TARGET.Arch == \"X86_64\" ) && ( TARGET.OpSys == \"LINUX\" ) && ( TARGET.Disk >= RequestDisk ) && ( TARGET.Memory >= RequestMemory ) && ( TARGET.HasFileTransfer );\n", "NumShadowStarts": 1, "MachineAttrCpus0": 1, "DAGManNodesLog": "/home/dentler/ChtcRun/project_auction/results_fix2/./mydag.dag.nodes.log", "Owner": "dentler", "Requirements": "undefined", "DiskUsage": 1250000, "JobLeaseDuration": 2400, "ClusterId": 49584018, "BufferSize": 524288, "IsCHTCSubmit": true, "RecentStatsLifetimeStarter": 48, "CommittedSlotTime": 58.0d, "Args": "--type=Matlab --version=R2011b --cmdtorun=net_est --unique=11021 --", "Environment": "", "LastPublicClaimId": "<128.105.245.39:54850>#1445038698#5043#...", "Iwd": "/home/dentler/ChtcRun/project_auction/results_fix2/11021", "QDate": 1446134012, "CurrentHosts": 0, "User": "dentler@chtc.wisc.edu", "StreamOut": false }
diff --git a/asterix-app/src/test/resources/runtimets/results/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.1.adm b/asterix-app/src/test/resources/runtimets/results/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.1.adm b/asterix-app/src/test/resources/runtimets/results/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 758c393..05f131e 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -2905,6 +2905,18 @@
             </compilation-unit>
         </test-case>
     </test-group>
+    <test-group name="meta">
+        <test-case FilePath="meta">
+            <compilation-unit name="query_dataset_with_meta-1">
+                <output-dir compare="Text">query_dataset_with_meta-1</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="meta">
+            <compilation-unit name="query_dataset_with_meta-2">
+                <output-dir compare="Text">query_dataset_with_meta-2</output-dir>
+            </compilation-unit>
+        </test-case>
+    </test-group>
     <test-group name="misc">
         <test-case FilePath="misc">
             <compilation-unit name="partition-by-nonexistent-field"> <!-- Seriously?? 3 expected errors -->
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
index 796be82..0d8a953 100644
--- a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
+++ b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
@@ -130,10 +130,37 @@
                             "Result for " + scriptFile + " changed at line " + num + ":\n< " + lineExpected + "\n> ");
                 }
 
-                if (!equalStrings(lineExpected.split("Time")[0], lineActual.split("Time")[0])) {
+                // Comparing result equality but ignore "Time"-prefixed fields. (for metadata tests.)
+                String[] lineSplitsExpected = lineExpected.split("Time");
+                String[] lineSplitsActual = lineActual.split("Time");
+                if (lineSplitsExpected.length != lineSplitsActual.length) {
                     throw new Exception("Result for " + scriptFile + " changed at line " + num + ":\n< " + lineExpected
                             + "\n> " + lineActual);
                 }
+                if (!equalStrings(lineSplitsExpected[0], lineSplitsActual[0])) {
+                    throw new Exception("Result for " + scriptFile + " changed at line " + num + ":\n< " + lineExpected
+                            + "\n> " + lineActual);
+                }
+
+                for (int i = 1; i < lineSplitsExpected.length; i++) {
+                    String[] splitsByCommaExpected = lineSplitsExpected[i].split(",");
+                    String[] splitsByCommaActual = lineSplitsActual[i].split(",");
+                    if (splitsByCommaExpected.length != splitsByCommaActual.length) {
+                        throw new Exception("Result for " + scriptFile + " changed at line " + num + ":\n< "
+                                + lineExpected + "\n> " + lineActual);
+                    }
+                    for (int j = 1; j < splitsByCommaExpected.length; j++) {
+                        if (splitsByCommaExpected[j].indexOf("DatasetId") >= 0) {
+                            // Ignore the field "DatasetId", which is different for different runs.
+                            // (for metadata tests)
+                            continue;
+                        }
+                        if (!equalStrings(splitsByCommaExpected[j], splitsByCommaActual[j])) {
+                            throw new Exception("Result for " + scriptFile + " changed at line " + num + ":\n< "
+                                    + lineExpected + "\n> " + lineActual);
+                        }
+                    }
+                }
 
                 ++num;
             }
@@ -156,8 +183,9 @@
             String row1 = rowsOne[i];
             String row2 = rowsTwo[i];
 
-            if (row1.equals(row2))
+            if (row1.equals(row2)) {
                 continue;
+            }
 
             String[] fields1 = row1.split(" ");
             String[] fields2 = row2.split(" ");
@@ -170,11 +198,11 @@
                 if (j >= fields2.length) {
                     return false;
                 } else if (fields1[j].equals(fields2[j])) {
-                    if (fields1[j].equals("{{"))
-                        bagEncountered = true;
+                    bagEncountered = fields1[j].equals("{{");
                     if (fields1[j].startsWith("}}")) {
-                        if (!bagElements1.equals(bagElements2))
+                        if (!bagElements1.equals(bagElements2)) {
                             return false;
+                        }
                         bagEncountered = false;
                         bagElements1.clear();
                         bagElements2.clear();
@@ -198,9 +226,9 @@
                         float float1 = (float) double1.doubleValue();
                         float float2 = (float) double2.doubleValue();
 
-                        if (Math.abs(float1 - float2) == 0)
+                        if (Math.abs(float1 - float2) == 0) {
                             continue;
-                        else {
+                        } else {
                             return false;
                         }
                     } catch (NumberFormatException ignored) {
@@ -698,8 +726,8 @@
                             break;
                         case "lib": // expected format <dataverse-name> <library-name>
                                     // <library-directory>
-                            // TODO: make this case work well with entity names containing spaces by
-                            // looking for \"
+                                    // TODO: make this case work well with entity names containing spaces by
+                                    // looking for \"
                             lines = statement.split("\n");
                             String lastLine = lines[lines.length - 1];
                             String[] command = lastLine.trim().split(" ");
diff --git a/asterix-lang-aql/src/main/javacc/AQL.html b/asterix-lang-aql/src/main/javacc/AQL.html
index b50d554..3da5e77 100644
--- a/asterix-lang-aql/src/main/javacc/AQL.html
+++ b/asterix-lang-aql/src/main/javacc/AQL.html
@@ -342,7 +342,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod21">DatasetSpecification</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE>( "external" &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod18">Identifier</A> &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> "using" <A HREF="#prod31">AdapterName</A> <A HREF="#prod32">Configuration</A> ( "on" <A HREF="#prod18">Identifier</A> )? ( "hints" <A HREF="#prod33">Properties</A> )? ( "using" "compaction" "policy" <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? | ( "internal" | "temporary" )? &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod18">Identifier</A> &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> <A HREF="#prod35">PrimaryKey</A> ( "autogenerated" )? ( "on" <A HREF="#prod18">Identifier</A> )? ( "hints" <A HREF="#prod33">Properties</A> )? ( "using" "compaction" "policy" <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? ( "with filter on" <A HREF="#prod36">NestedField</A> )? )</TD>
+<TD ALIGN=LEFT VALIGN=BASELINE>( "external" &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> "using" <A HREF="#prod31">AdapterName</A> <A HREF="#prod32">Configuration</A> ( "on" <A HREF="#prod18">Identifier</A> )? ( "hints" <A HREF="#prod33">Properties</A> )? ( "using" "compaction" "policy" <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? | ( "internal" | "temporary" )? &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> ( &lt;COMMA&gt; <A HREF="#prod27">TypeName</A> )? &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> <A HREF="#prod35">PrimaryKey</A> ( "autogenerated" )? ( "on" <A HREF="#prod18">Identifier</A> )? ( "hints" <A HREF="#prod33">Properties</A> )? ( "using" "compaction" "policy" <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? ( "with filter on" <A HREF="#prod36">NestedField</A> )? )</TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod16">RefreshExternalDatasetStatement</A></TD>
@@ -437,7 +437,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod10">InsertStatement</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE>"insert" "into" &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> <A HREF="#prod15">Query</A></TD>
+<TD ALIGN=LEFT VALIGN=BASELINE>( "insert" | "upsert" ) "into" &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> <A HREF="#prod15">Query</A></TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod11">DeleteStatement</A></TD>
@@ -562,7 +562,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod36">NestedField</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE><A HREF="#prod18">Identifier</A> ( &lt;DOT&gt; <A HREF="#prod18">Identifier</A> )*</TD>
+<TD ALIGN=LEFT VALIGN=BASELINE>( "@" )? <A HREF="#prod18">Identifier</A> ( &lt;DOT&gt; <A HREF="#prod18">Identifier</A> )*</TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod40">StringLiteral</A></TD>
diff --git a/asterix-lang-aql/src/main/javacc/AQL.jj b/asterix-lang-aql/src/main/javacc/AQL.jj
index 93e3f68..ea95871 100644
--- a/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -416,6 +416,7 @@
   String compactionPolicy = null;
   boolean temp = false;
   List<String> filterField = null;
+  Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
 }
 {
   (
@@ -434,6 +435,8 @@
                                    nameComponents.second,
                                    typeComponents.first,
                                    typeComponents.second,
+                                   metaTypeComponents.first,
+                                   metaTypeComponents.second,
                                    nodeGroupName != null? new Identifier(nodeGroupName): null,
                                    compactionPolicy,
                                    compactionPolicyProperties,
@@ -446,9 +449,20 @@
     | ("internal" | "temporary" {
             temp = token.image.toLowerCase().equals("temporary");
         }
-      )? 
+      )?
     <DATASET> nameComponents = QualifiedName()
     <LEFTPAREN> typeComponents = TypeName() <RIGHTPAREN>
+    (
+        { String name; }
+        <WITH>
+        name = Identifier()
+        {
+            if(!name.equals("meta")){
+                throw new ParseException("We can only support one additional associated field called \"meta\".");
+            }
+        }
+        <LEFTPAREN> metaTypeComponents = TypeName() <RIGHTPAREN>
+    )?
     ifNotExists = IfNotExists()
     primaryKeyFields = PrimaryKey()
     ("autogenerated" { autogenerated = true; } )?
@@ -465,6 +479,8 @@
                                    nameComponents.second,
                                    typeComponents.first,
                                    typeComponents.second,
+                                   metaTypeComponents.first,
+                                   metaTypeComponents.second,
                                    nodeGroupName != null ? new Identifier(nodeGroupName) : null,
                                    compactionPolicy,
                                    compactionPolicyProperties,
@@ -1386,7 +1402,23 @@
 {
   lit = Identifier()
   {
-    exprList.add(lit);
+    boolean meetParens = false;
+  }
+  (
+    <LEFTPAREN><RIGHTPAREN>
+    {
+        if(lit.equals("meta")){
+            exprList.add(lit+"()");
+        }else{
+            throw new ParseException("The string before () has to be \"meta\".");
+        }
+        meetParens = true;
+    }
+  )?
+  {
+    if(!meetParens){
+        exprList.add(lit);
+    }
   }
   (<DOT>
     lit = Identifier()
diff --git a/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java b/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
index d9d09f7..10cf8db 100644
--- a/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
+++ b/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
@@ -20,8 +20,8 @@
 
 import java.util.Map;
 
-import org.apache.asterix.common.config.MetadataConstants;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.config.MetadataConstants;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -40,9 +40,13 @@
     protected final Map<String, String> hints;
     protected final boolean ifNotExists;
 
+    protected final Identifier metaItemTypeDataverse;
+    protected final Identifier metaItemTypeName;
+
     public DatasetDecl(Identifier dataverse, Identifier name, Identifier itemTypeDataverse, Identifier itemTypeName,
-            Identifier nodeGroupName, String compactionPolicy, Map<String, String> compactionPolicyProperties,
-            Map<String, String> hints, DatasetType datasetType, IDatasetDetailsDecl idd, boolean ifNotExists) {
+            Identifier metaItemTypeDataverse, Identifier metaItemTypeName, Identifier nodeGroupName,
+            String compactionPolicy, Map<String, String> compactionPolicyProperties, Map<String, String> hints,
+            DatasetType datasetType, IDatasetDetailsDecl idd, boolean ifNotExists) {
         this.dataverse = dataverse;
         this.name = name;
         this.itemTypeName = itemTypeName;
@@ -51,6 +55,12 @@
         } else {
             this.itemTypeDataverse = itemTypeDataverse;
         }
+        this.metaItemTypeName = metaItemTypeName;
+        if (metaItemTypeDataverse == null || metaItemTypeDataverse.getValue() == null) {
+            this.metaItemTypeDataverse = dataverse;
+        } else {
+            this.metaItemTypeDataverse = metaItemTypeDataverse;
+        }
         this.nodegroupName = nodeGroupName == null ? new Identifier(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)
                 : nodeGroupName;
         this.compactionPolicy = compactionPolicy;
@@ -89,6 +99,26 @@
         }
     }
 
+    public Identifier getMetaName() {
+        return name;
+    }
+
+    public Identifier getMetaItemTypeName() {
+        return metaItemTypeName == null ? new Identifier() : metaItemTypeName;
+    }
+
+    public Identifier getMetaItemTypeDataverse() {
+        return metaItemTypeDataverse == null ? new Identifier() : metaItemTypeDataverse;
+    }
+
+    public String getQualifiedMetaTypeName() {
+        if (metaItemTypeDataverse == dataverse) {
+            return metaItemTypeName.getValue();
+        } else {
+            return metaItemTypeDataverse.getValue() + "." + metaItemTypeName.getValue();
+        }
+    }
+
     public Identifier getNodegroupName() {
         return nodegroupName;
     }
diff --git a/asterix-lang-sqlpp/src/main/javacc/SQLPP.html b/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
index 4bb64d2..42aaa06 100644
--- a/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
+++ b/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
@@ -403,7 +403,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod21">DatasetSpecification</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE>( &lt;EXTERNAL&gt; &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod18">Identifier</A> &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> &lt;USING&gt; <A HREF="#prod31">AdapterName</A> <A HREF="#prod32">Configuration</A> ( &lt;ON&gt; <A HREF="#prod18">Identifier</A> )? ( &lt;HINTS&gt; <A HREF="#prod33">Properties</A> )? ( &lt;USING&gt; &lt;COMPACTION&gt; &lt;POLICY&gt; <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? | ( &lt;INTERNAL&gt; | &lt;TEMPORARY&gt; )? &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod18">Identifier</A> &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> <A HREF="#prod35">PrimaryKey</A> ( &lt;AUTOGENERATED&gt; )? ( &lt;ON&gt; <A HREF="#prod18">Identifier</A> )? ( &lt;HINTS&gt; <A HREF="#prod33">Properties</A> )? ( &lt;USING&gt; &lt;COMPACTION&gt; &lt;POLICY&gt; <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? ( &lt;WITH&gt; &lt;FILTER&gt; &lt;ON&gt; <A HREF="#prod36">NestedField</A> )? )</TD>
+<TD ALIGN=LEFT VALIGN=BASELINE>( &lt;EXTERNAL&gt; &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> &lt;USING&gt; <A HREF="#prod31">AdapterName</A> <A HREF="#prod32">Configuration</A> ( &lt;ON&gt; <A HREF="#prod18">Identifier</A> )? ( &lt;HINTS&gt; <A HREF="#prod33">Properties</A> )? ( &lt;USING&gt; &lt;COMPACTION&gt; &lt;POLICY&gt; <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? | ( &lt;INTERNAL&gt; | &lt;TEMPORARY&gt; )? &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> ( &lt;COMMA&gt; <A HREF="#prod27">TypeName</A> )? &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> <A HREF="#prod35">PrimaryKey</A> ( &lt;AUTOGENERATED&gt; )? ( &lt;ON&gt; <A HREF="#prod18">Identifier</A> )? ( &lt;HINTS&gt; <A HREF="#prod33">Properties</A> )? ( &lt;USING&gt; &lt;COMPACTION&gt; &lt;POLICY&gt; <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? ( &lt;WITH&gt; &lt;FILTER&gt; &lt;ON&gt; <A HREF="#prod36">NestedField</A> )? )</TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod16">RefreshExternalDatasetStatement</A></TD>
@@ -623,7 +623,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod36">NestedField</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE><A HREF="#prod18">Identifier</A> ( &lt;DOT&gt; <A HREF="#prod18">Identifier</A> )*</TD>
+<TD ALIGN=LEFT VALIGN=BASELINE>( &lt;ATT&gt; )? <A HREF="#prod18">Identifier</A> ( &lt;DOT&gt; <A HREF="#prod18">Identifier</A> )*</TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod40">QuotedString</A></TD>
diff --git a/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 547a10b..6259faa 100644
--- a/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -429,6 +429,7 @@
   String compactionPolicy = null;
   boolean temp = false;
   List<String> filterField = null;
+  Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
 }
 {
   (
@@ -447,6 +448,8 @@
                                    nameComponents.second,
                                    typeComponents.first,
                                    typeComponents.second,
+                                   metaTypeComponents.first,
+                                   metaTypeComponents.second,
                                    nodeGroupName != null? new Identifier(nodeGroupName): null,
                                    compactionPolicy,
                                    compactionPolicyProperties,
@@ -456,9 +459,20 @@
                                    ifNotExists);
       }
 
-    | (<INTERNAL> | <TEMPORARY> { temp = true; })? 
+    | (<INTERNAL> | <TEMPORARY> { temp = true; })?
     <DATASET> nameComponents = QualifiedName()
     <LEFTPAREN> typeComponents = TypeName() <RIGHTPAREN>
+    (
+        { String name; }
+        <WITH>
+        name = Identifier()
+        {
+            if(!name.toLowerCase().equals("meta")){
+                throw new ParseException("We can only support one additional associated field called \"meta\".");
+            }
+        }
+        <LEFTPAREN> metaTypeComponents = TypeName() <RIGHTPAREN>
+    )?
     ifNotExists = IfNotExists()
     primaryKeyFields = PrimaryKey()
     (<AUTOGENERATED> { autogenerated = true; } )?
@@ -475,6 +489,8 @@
                                    nameComponents.second,
                                    typeComponents.first,
                                    typeComponents.second,
+                                   metaTypeComponents.first,
+                                   metaTypeComponents.second,
                                    nodeGroupName != null ? new Identifier(nodeGroupName) : null,
                                    compactionPolicy,
                                    compactionPolicyProperties,
@@ -1385,7 +1401,23 @@
 {
   lit = Identifier()
   {
-    exprList.add(lit);
+    boolean meetParens = false;
+  }
+  (
+    <LEFTPAREN><RIGHTPAREN>
+    {
+        if(lit.toLowerCase().equals("meta")){
+            exprList.add(lit.toLowerCase() + "()");
+        }else{
+            throw new ParseException("The string before () has to be \"meta\".");
+        }
+        meetParens = true;
+    }
+  )?
+  {
+    if(!meetParens){
+        exprList.add(lit);
+    }
   }
   (<DOT>
     lit = Identifier()
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 44a586a..55a5f3d 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -249,6 +249,8 @@
     public static final int DATASET_ARECORD_TIMESTAMP_FIELD_INDEX = 11;
     public static final int DATASET_ARECORD_DATASETID_FIELD_INDEX = 12;
     public static final int DATASET_ARECORD_PENDINGOP_FIELD_INDEX = 13;
+    public static final String DATASET_ARECORD_METATYPEDATAVERSENAME_FIELD_NAME = "MetatypeDataverseName";
+    public static final String DATASET_ARECORD_METATYPENAME_FIELD_NAME = "MetatypeName";
 
     private static final ARecordType createDatasetRecordType() throws AsterixException {
         String[] fieldNames = { "DataverseName", "DatasetName", "DatatypeDataverseName", "DatatypeName", "DatasetType",
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlDataSource.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlDataSource.java
index 5ad0c52..a10e191 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlDataSource.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlDataSource.java
@@ -48,6 +48,7 @@
 
     private final AqlSourceId id;
     private final IAType itemType;
+    private final IAType metaItemType;
     private final AqlDataSourceType datasourceType;
     protected IAType[] schemaTypes;
     protected INodeDomain domain;
@@ -60,9 +61,11 @@
         LOADABLE
     }
 
-    public AqlDataSource(AqlSourceId id, IAType itemType, AqlDataSourceType datasourceType) throws AlgebricksException {
+    public AqlDataSource(AqlSourceId id, IAType itemType, IAType metaItemType, AqlDataSourceType datasourceType)
+            throws AlgebricksException {
         this.id = id;
         this.itemType = itemType;
+        this.metaItemType = metaItemType;
         this.datasourceType = datasourceType;
     }
 
@@ -194,6 +197,14 @@
         return itemType;
     }
 
+    public IAType getMetaItemType() {
+        return metaItemType;
+    }
+
+    public boolean hasMeta() {
+        return metaItemType != null;
+    }
+
     public void setProperties(Map<String, Serializable> properties) {
         this.properties = properties;
     }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
index 9764709..7ce7ba3 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
@@ -642,9 +642,8 @@
             ITypeTraits[] typeTraits;
             IBinaryComparatorFactory[] comparatorFactories;
 
-            String itemTypeName = dataset.getItemTypeName();
-            ARecordType itemType = (ARecordType) MetadataManager.INSTANCE
-                    .getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
+            ARecordType itemType = (ARecordType) this.findType(dataset.getItemTypeDataverseName(),
+                    dataset.getItemTypeName());
             ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
             IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
                     itemType, context.getBinaryComparatorFactoryProvider());
@@ -985,12 +984,11 @@
         if (dataset == null) {
             throw new AlgebricksException("Datasource with id " + aqlId + " was not found.");
         }
-        String tName = dataset.getItemTypeName();
-        IAType itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), tName)
-                .getDatatype();
+        IAType itemType = findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        IAType metaItemType = findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
         AqlDataSourceType datasourceType = dataset.getDatasetType().equals(DatasetType.EXTERNAL)
                 ? AqlDataSourceType.EXTERNAL_DATASET : AqlDataSourceType.INTERNAL_DATASET;
-        return new DatasetDataSource(aqlId, aqlId.getDataverseName(), aqlId.getDatasourceName(), itemType,
+        return new DatasetDataSource(aqlId, aqlId.getDataverseName(), aqlId.getDatasourceName(), itemType, metaItemType,
                 datasourceType);
     }
 
@@ -2126,6 +2124,9 @@
     }
 
     public IAType findType(String dataverse, String typeName) throws AlgebricksException {
+        if (dataverse == null || typeName == null) {
+            return null;
+        }
         Datatype type;
         try {
             type = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverse, typeName);
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index 3c76287..8b5956f 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -28,9 +28,7 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
-import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 
 public class DatasetDataSource extends AqlDataSource {
@@ -38,20 +36,20 @@
     private Dataset dataset;
 
     public DatasetDataSource(AqlSourceId id, String datasourceDataverse, String datasourceName, IAType itemType,
-            AqlDataSourceType datasourceType) throws AlgebricksException {
-        super(id, itemType, datasourceType);
+            IAType metaItemType, AqlDataSourceType datasourceType) throws AlgebricksException {
+        super(id, itemType, metaItemType, datasourceType);
         MetadataTransactionContext ctx = null;
         try {
             ctx = MetadataManager.INSTANCE.beginTransaction();
             dataset = MetadataManager.INSTANCE.getDataset(ctx, id.getDataverseName(), id.getDatasourceName());
             if (dataset == null) {
-                throw new AlgebricksException("Unknown dataset " + datasourceName + " in dataverse "
-                        + datasourceDataverse);
+                throw new AlgebricksException(
+                        "Unknown dataset " + datasourceName + " in dataverse " + datasourceDataverse);
             }
             MetadataManager.INSTANCE.commitTransaction(ctx);
             switch (dataset.getDatasetType()) {
                 case INTERNAL:
-                    initInternalDataset(itemType);
+                    initInternalDataset(itemType, metaItemType);
                     break;
                 case EXTERNAL:
                     initExternalDataset(itemType);
@@ -76,15 +74,18 @@
         return dataset;
     }
 
-    private void initInternalDataset(IAType itemType) throws IOException, AlgebricksException {
+    private void initInternalDataset(IAType itemType, IAType metaItemType) throws IOException, AlgebricksException {
         List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
         ARecordType recordType = (ARecordType) itemType;
         int n = partitioningKeys.size();
-        schemaTypes = new IAType[n + 1];
+        schemaTypes = metaItemType == null ? new IAType[n + 1] : new IAType[n + 2];
         for (int i = 0; i < n; i++) {
             schemaTypes[i] = recordType.getSubFieldType(partitioningKeys.get(i));
         }
         schemaTypes[n] = itemType;
+        if (metaItemType != null) {
+            schemaTypes[n + 1] = metaItemType;
+        }
         domain = new DefaultNodeGroupDomain(dataset.getNodeGroupName());
     }
 
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index 6f540df..46e3007 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -39,10 +39,10 @@
     private final String[] locations;
     private final int computeCardinality;
 
-    public FeedDataSource(AqlSourceId id, String targetDataset, IAType itemType, AqlDataSourceType dataSourceType,
-            FeedId sourceFeedId, IFeed.FeedType sourceFeedType, ConnectionLocation location, String[] locations)
-                    throws AlgebricksException {
-        super(id, itemType, dataSourceType);
+    public FeedDataSource(AqlSourceId id, String targetDataset, IAType itemType, IAType metaItemType,
+            AqlDataSourceType dataSourceType, FeedId sourceFeedId, IFeed.FeedType sourceFeedType,
+            ConnectionLocation location, String[] locations) throws AlgebricksException {
+        super(id, itemType, metaItemType, dataSourceType);
         this.targetDataset = targetDataset;
         this.sourceFeedId = sourceFeedId;
         this.sourceFeedType = sourceFeedType;
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
index 74bd5e4..8c8e92b 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
@@ -32,10 +32,6 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
-import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
-import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class LoadableDataSource extends AqlDataSource {
@@ -46,9 +42,9 @@
     private final Map<String, String> adapterProperties;
     private final boolean isPKAutoGenerated;
 
-    public LoadableDataSource(Dataset targetDataset, IAType itemType, String adapter, Map<String, String> properties)
-            throws AlgebricksException, IOException {
-        super(new AqlSourceId("loadable_dv", "loadable_ds"), itemType, AqlDataSourceType.LOADABLE);
+    public LoadableDataSource(Dataset targetDataset, IAType itemType, IAType metaItemType, String adapter,
+            Map<String, String> properties) throws AlgebricksException, IOException {
+        super(new AqlSourceId("loadable_dv", "loadable_ds"), itemType, metaItemType, AqlDataSourceType.LOADABLE);
         this.targetDataset = targetDataset;
         this.adapter = adapter;
         this.adapterProperties = properties;
@@ -76,9 +72,10 @@
             IAType fieldType = null;
             if (partitioningKeys.get(0).equals(recType.getFieldNames()[j])) {
                 if (recType.getFieldTypes()[j].getTypeTag() == ATypeTag.RECORD) {
-                    if (j != 0)
+                    if (j != 0) {
                         throw new AsterixException("Autogenerated key " + StringUtils.join(partitioningKeys, '.')
                                 + " should be a first field of the type " + recType.getTypeName());
+                    }
                     partitioningKeys.remove(0);
                     fieldType = getStrippedPKType(partitioningKeys, (ARecordType) recType.getFieldTypes()[j]);
                 } else {
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 64866e1..aba8f8d 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -51,14 +51,29 @@
     // Type of pending operations with respect to atomic DDL operation
     private int pendingOp;
 
+    // Dataverse of Meta ItemType for this dataset.
+    private final String metaItemTypeDataverseName;
+    // Type of Meta items stored in this dataset.
+    private final String metaItemTypeName;
+
     public Dataset(String dataverseName, String datasetName, String itemTypeDataverseName, String itemTypeName,
             String nodeGroupName, String compactionPolicy, Map<String, String> compactionPolicyProperties,
             IDatasetDetails datasetDetails, Map<String, String> hints, DatasetType datasetType, int datasetId,
             int pendingOp) {
+        this(dataverseName, datasetName, itemTypeDataverseName, itemTypeName, null, null, nodeGroupName,
+                compactionPolicy, compactionPolicyProperties, datasetDetails, hints, datasetType, datasetId, pendingOp);
+    }
+
+    public Dataset(String dataverseName, String datasetName, String itemTypeDataverseName, String itemTypeName,
+            String metaItemTypeDataverseName, String metaItemTypeName, String nodeGroupName, String compactionPolicy,
+            Map<String, String> compactionPolicyProperties, IDatasetDetails datasetDetails, Map<String, String> hints,
+            DatasetType datasetType, int datasetId, int pendingOp) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
         this.itemTypeName = itemTypeName;
         this.itemTypeDataverseName = itemTypeDataverseName;
+        this.metaItemTypeDataverseName = metaItemTypeDataverseName;
+        this.metaItemTypeName = metaItemTypeName;
         this.nodeGroupName = nodeGroupName;
         this.compactionPolicy = compactionPolicy;
         this.compactionPolicyProperties = compactionPolicyProperties;
@@ -117,6 +132,18 @@
         return pendingOp;
     }
 
+    public String getMetaItemTypeDataverseName() {
+        return metaItemTypeDataverseName;
+    }
+
+    public String getMetaItemTypeName() {
+        return metaItemTypeName;
+    }
+
+    public boolean hasMetaPart() {
+        return metaItemTypeDataverseName != null && metaItemTypeName != null;
+    }
+
     public void setPendingOp(int pendingOp) {
         this.pendingOp = pendingOp;
     }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index f283a83..12543c3 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -84,7 +84,8 @@
     private final ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(MetadataRecordTypes.DATASET_RECORDTYPE);
     private final AMutableInt32 aInt32;
-    protected ISerializerDeserializer<AInt32> aInt32Serde;
+    protected final ISerializerDeserializer<AInt32> aInt32Serde;
+    private final ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage();
 
     @SuppressWarnings("unchecked")
     public DatasetTupleTranslator(boolean getTuple) {
@@ -223,8 +224,21 @@
 
         Map<String, String> hints = getDatasetHints(datasetRecord);
 
-        return new Dataset(dataverseName, datasetName, typeDataverseName, typeName, nodeGroupName, compactionPolicy,
-                compactionPolicyProperties, datasetDetails, hints, datasetType, datasetId, pendingOp);
+        String metaTypeDataverseName = null;
+        String metaTypeName = null;
+        int metaTypeDataverseNameIndex = datasetRecord.getType()
+                .getFieldIndex(MetadataRecordTypes.DATASET_ARECORD_METATYPEDATAVERSENAME_FIELD_NAME);
+        if (metaTypeDataverseNameIndex >= 0) {
+            metaTypeDataverseName = ((AString) datasetRecord.getValueByPos(metaTypeDataverseNameIndex))
+                    .getStringValue();
+            int metaTypeNameIndex = datasetRecord.getType()
+                    .getFieldIndex(MetadataRecordTypes.DATASET_ARECORD_METATYPENAME_FIELD_NAME);
+            metaTypeName = ((AString) datasetRecord.getValueByPos(metaTypeNameIndex)).getStringValue();
+        }
+
+        return new Dataset(dataverseName, datasetName, typeDataverseName, typeName, metaTypeDataverseName, metaTypeName,
+                nodeGroupName, compactionPolicy, compactionPolicyProperties, datasetDetails, hints, datasetType,
+                datasetId, pendingOp);
     }
 
     @Override
@@ -341,6 +355,27 @@
         aInt32Serde.serialize(aInt32, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_PENDINGOP_FIELD_INDEX, fieldValue);
 
+        // write open fields
+        if (dataset.hasMetaPart()) {
+            // write open field 1, the meta item type Dataverse name.
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.DATASET_ARECORD_METATYPEDATAVERSENAME_FIELD_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(dataset.getMetaItemTypeDataverseName());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            recordBuilder.addField(fieldName, fieldValue);
+
+            // write open field 2, the meta item type name.
+            fieldName.reset();
+            aString.setValue(MetadataRecordTypes.DATASET_ARECORD_METATYPENAME_FIELD_NAME);
+            stringSerde.serialize(aString, fieldName.getDataOutput());
+            fieldValue.reset();
+            aString.setValue(dataset.getMetaItemTypeName());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+            recordBuilder.addField(fieldName, fieldValue);
+        }
+
         // write record
         recordBuilder.write(tupleBuilder.getDataOutput(), true);
         tupleBuilder.addFieldEndOffset();
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java
index 9919962..19471de 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtils.java
@@ -218,7 +218,8 @@
         }
 
         List<List<String>> partitioningKeys = getPartitioningKeys(dataset);
-        int[] btreeFields = new int[partitioningKeys.size() + 1];
+        int valueFields = dataset.hasMetaPart() ? 2 : 1;
+        int[] btreeFields = new int[partitioningKeys.size() + valueFields];
         for (int i = 0; i < btreeFields.length; ++i) {
             btreeFields[i] = i;
         }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java
index 03fc4c0..5cecd84 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockManager.java
@@ -292,13 +292,22 @@
     }
 
     public void createDatasetBegin(String dataverseName, String itemTypeDataverseName,
-            String itemTypeFullyQualifiedName, String nodeGroupName, String compactionPolicyName,
-            String datasetFullyQualifiedName, boolean isDefaultCompactionPolicy) {
+            String itemTypeFullyQualifiedName, String metaItemTypeDataverseName, String metaItemTypeFullyQualifiedName,
+            String nodeGroupName, String compactionPolicyName, String datasetFullyQualifiedName,
+            boolean isDefaultCompactionPolicy) {
         acquireDataverseReadLock(dataverseName);
         if (!dataverseName.equals(itemTypeDataverseName)) {
             acquireDataverseReadLock(itemTypeDataverseName);
         }
+        if (metaItemTypeDataverseName != null && !metaItemTypeDataverseName.equals(dataverseName)
+                && !metaItemTypeDataverseName.equals(itemTypeDataverseName)) {
+            acquireDataverseReadLock(metaItemTypeDataverseName);
+        }
         acquireDataTypeReadLock(itemTypeFullyQualifiedName);
+        if (metaItemTypeFullyQualifiedName != null
+                && !metaItemTypeFullyQualifiedName.equals(itemTypeFullyQualifiedName)) {
+            acquireDataTypeReadLock(metaItemTypeFullyQualifiedName);
+        }
         acquireNodeGroupReadLock(nodeGroupName);
         if (!isDefaultCompactionPolicy) {
             acquireCompactionPolicyReadLock(compactionPolicyName);
@@ -307,14 +316,22 @@
     }
 
     public void createDatasetEnd(String dataverseName, String itemTypeDataverseName, String itemTypeFullyQualifiedName,
-            String nodeGroupName, String compactionPolicyName, String datasetFullyQualifiedName,
-            boolean isDefaultCompactionPolicy) {
+            String metaItemTypeDataverseName, String metaItemTypeFullyQualifiedName, String nodeGroupName,
+            String compactionPolicyName, String datasetFullyQualifiedName, boolean isDefaultCompactionPolicy) {
         releaseDatasetWriteLock(datasetFullyQualifiedName);
         if (!isDefaultCompactionPolicy) {
             releaseCompactionPolicyReadLock(compactionPolicyName);
         }
         releaseNodeGroupReadLock(nodeGroupName);
+        if (metaItemTypeFullyQualifiedName != null
+                && !metaItemTypeFullyQualifiedName.equals(itemTypeFullyQualifiedName)) {
+            releaseDataTypeReadLock(metaItemTypeFullyQualifiedName);
+        }
         releaseDataTypeReadLock(itemTypeFullyQualifiedName);
+        if (metaItemTypeDataverseName != null && !metaItemTypeDataverseName.equals(dataverseName)
+                && !metaItemTypeDataverseName.equals(itemTypeDataverseName)) {
+            releaseDataverseReadLock(metaItemTypeDataverseName);
+        }
         if (!dataverseName.equals(itemTypeDataverseName)) {
             releaseDataverseReadLock(itemTypeDataverseName);
         }
diff --git a/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java b/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
new file mode 100644
index 0000000..9328d30
--- /dev/null
+++ b/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.metadata.entitytupletranslators;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.metadata.MetadataException;
+import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class DatasetTupleTranslatorTest {
+
+    @Test
+    public void test() throws MetadataException, IOException {
+        Map<String, String> compactionPolicyProperties = new HashMap<>();
+        compactionPolicyProperties.put("max-mergable-component-size", "1073741824");
+        compactionPolicyProperties.put("max-tolerance-component-count", "3");
+
+        InternalDatasetDetails details = new InternalDatasetDetails(FileStructure.BTREE, PartitioningStrategy.HASH,
+                Collections.singletonList(Collections.singletonList("row_id")),
+                Collections.singletonList(Collections.singletonList("row_id")),
+                Collections.singletonList(BuiltinType.AINT64), false, Collections.emptyList(), false);
+
+        Dataset dataset = new Dataset("test", "log", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES",
+                "prefix", compactionPolicyProperties, details, Collections.emptyMap(), DatasetType.INTERNAL, 115, 0);
+
+        MetadataRecordTypes.init();
+        MetadataPrimaryIndexes.init();
+        DatasetTupleTranslator dtTranslator = new DatasetTupleTranslator(true);
+        ITupleReference tuple = dtTranslator.getTupleFromMetadataEntity(dataset);
+        Dataset deserializedDataset = dtTranslator.getMetadataEntityFromTuple(tuple);
+        Assert.assertEquals(dataset.getMetaItemTypeDataverseName(), deserializedDataset.getMetaItemTypeDataverseName());
+        Assert.assertEquals(dataset.getMetaItemTypeName(), deserializedDataset.getMetaItemTypeName());
+    }
+}