merge from master
diff --git a/.gitignore b/.gitignore
index 85e5801..af99107 100644
--- a/.gitignore
+++ b/.gitignore
@@ -12,5 +12,7 @@
asterix-app/rttest
asterix-app/mdtest/
asterix-app/opttest/
+asterix-installer/ittest/
build
asterix_logs
+bin/
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java
new file mode 100644
index 0000000..a8e8049
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.algebra.operators;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractExtensibleLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorExtension;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+
+public class MaterializeOperator extends AbstractExtensibleLogicalOperator {
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public IOperatorExtension newInstance() {
+ return new MaterializeOperator();
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "materialize";
+ }
+
+ @Override
+ public void getUsedVariables(Collection<LogicalVariable> usedVars) {
+ // No used variables.
+ }
+
+ @Override
+ public void getProducedVariables(Collection<LogicalVariable> producedVars) {
+ // No produced variables.
+ }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
index 269526b..32bfa58 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
@@ -19,11 +19,11 @@
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -35,16 +35,18 @@
import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
public class CommitRuntime implements IPushRuntime {
-
+
private final static long SEED = 0L;
private final IHyracksTaskContext hyracksTaskCtx;
private final ITransactionManager transactionManager;
+ private final ILogManager logMgr;
private final JobId jobId;
- private final DatasetId datasetId;
+ private final int datasetId;
private final int[] primaryKeyFields;
private final boolean isWriteTransaction;
- private final long[] longHashes;
+ private final long[] longHashes;
+ private final LogRecord logRecord;
private ITransactionContext transactionContext;
private RecordDescriptor inputRecordDesc;
@@ -54,23 +56,24 @@
public CommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
boolean isWriteTransaction) {
this.hyracksTaskCtx = ctx;
- IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
- .getApplicationObject();
+ IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject();
this.transactionManager = runtimeCtx.getTransactionSubsystem().getTransactionManager();
+ this.logMgr = runtimeCtx.getTransactionSubsystem().getLogManager();
this.jobId = jobId;
- this.datasetId = new DatasetId(datasetId);
+ this.datasetId = datasetId;
this.primaryKeyFields = primaryKeyFields;
this.frameTupleReference = new FrameTupleReference();
this.isWriteTransaction = isWriteTransaction;
- this.longHashes= new long[2];
+ this.longHashes = new long[2];
+ this.logRecord = new LogRecord();
}
@Override
public void open() throws HyracksDataException {
try {
- transactionContext = transactionManager.getTransactionContext(jobId);
- transactionContext.setTransactionType(isWriteTransaction ? TransactionType.READ_WRITE
- : TransactionType.READ);
+ transactionContext = transactionManager.getTransactionContext(jobId, false);
+ transactionContext.setWriteTxn(isWriteTransaction);
} catch (ACIDException e) {
throw new HyracksDataException(e);
}
@@ -84,17 +87,19 @@
for (int t = 0; t < nTuple; t++) {
frameTupleReference.reset(frameTupleAccessor, t);
pkHash = computePrimaryKeyHashValue(frameTupleReference, primaryKeyFields);
+ logRecord.formEntityCommitLogRecord(transactionContext, datasetId, pkHash, frameTupleReference,
+ primaryKeyFields);
try {
- transactionManager.commitTransaction(transactionContext, datasetId, pkHash);
+ logMgr.log(logRecord);
} catch (ACIDException e) {
throw new HyracksDataException(e);
}
}
}
-
+
private int computePrimaryKeyHashValue(ITupleReference tuple, int[] primaryKeyFields) {
MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
- return Math.abs((int) longHashes[0]);
+ return Math.abs((int) longHashes[0]);
}
@Override
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 23f1d56..af503a6 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -20,6 +20,7 @@
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
@@ -38,6 +39,7 @@
import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod.SearchModifierType;
import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexJobGenParams;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -217,18 +219,16 @@
if (!isPartitioned) {
dataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate());
} else {
dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate());
}
LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java
new file mode 100644
index 0000000..dc25ae7
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor;
+
+public class MaterializePOperator extends AbstractPhysicalOperator {
+
+ private final boolean isSingleActivity;
+
+ public MaterializePOperator(boolean isSingleActivity) {
+ this.isSingleActivity = isSingleActivity;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.EXTENSION_OPERATOR;
+ }
+
+ @Override
+ public String toString() {
+ return "MATERIALIZE";
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
+ propagatedSchema, context);
+ MaterializingOperatorDescriptor materializationOpDesc = new MaterializingOperatorDescriptor(
+ builder.getJobSpec(), recDescriptor, isSingleActivity);
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, materializationOpDesc);
+ ILogicalOperator src = op.getInputs().get(0).getValue();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index c3c50b4..6b4cdb1 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -32,7 +32,8 @@
import edu.uci.ics.asterix.optimizer.rules.IntroduceDynamicTypeCastRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceEnforcedListTypeRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceInstantLockSearchCallbackRule;
-import edu.uci.ics.asterix.optimizer.rules.IntroduceRapidFrameFlushProjectRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceMaterializationForInsertWithSelfScanRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceRapidFrameFlushProjectAssignRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastForInsertRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceUnnestForCollectionToSequenceRule;
@@ -63,6 +64,7 @@
import edu.uci.ics.hyracks.algebricks.rewriter.rules.ComplexUnnestToProductRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateAssignsRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateGroupByEmptyKeyRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateSubplanRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.EnforceOrderByAfterSubplan;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule;
@@ -82,6 +84,7 @@
import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByForSubplanRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceProjectsRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.LeftOuterJoinToInnerJoinRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushAssignBelowUnionAllRule;
import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushAssignDownThroughProductRule;
@@ -162,6 +165,8 @@
condPushDownAndJoinInference.add(new PushProperJoinThroughProduct());
condPushDownAndJoinInference.add(new PushGroupByThroughProduct());
condPushDownAndJoinInference.add(new NestGroupByRule());
+ condPushDownAndJoinInference.add(new EliminateGroupByEmptyKeyRule());
+ condPushDownAndJoinInference.add(new LeftOuterJoinToInnerJoinRule());
return condPushDownAndJoinInference;
}
@@ -246,6 +251,7 @@
physicalRewritesAllLevels.add(new PullPositionalVariableFromUnnestRule());
physicalRewritesAllLevels.add(new PushProjectDownRule());
physicalRewritesAllLevels.add(new InsertProjectBeforeUnionRule());
+ physicalRewritesAllLevels.add(new IntroduceMaterializationForInsertWithSelfScanRule());
physicalRewritesAllLevels.add(new InlineSingleReferenceVariablesRule());
physicalRewritesAllLevels.add(new RemoveUnusedAssignAndAggregateRule());
physicalRewritesAllLevels.add(new ConsolidateAssignsRule());
@@ -260,7 +266,7 @@
physicalRewritesTopLevel.add(new PushLimitDownRule());
physicalRewritesTopLevel.add(new IntroduceProjectsRule());
physicalRewritesTopLevel.add(new SetAlgebricksPhysicalOperatorsRule());
- physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectRule());
+ physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectAssignRule());
physicalRewritesTopLevel.add(new SetExecutionModeRule());
return physicalRewritesTopLevel;
}
@@ -273,6 +279,7 @@
// Re-infer all types, so that, e.g., the effect of not-is-null is
// propagated.
prepareForJobGenRewrites.add(new ReinferAllTypesRule());
+ prepareForJobGenRewrites.add(new SetExecutionModeRule());
return prepareForJobGenRewrites;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
new file mode 100644
index 0000000..c15b0f0
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.optimizer.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.algebra.operators.MaterializeOperator;
+import edu.uci.ics.asterix.algebra.operators.physical.MaterializePOperator;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.optimizer.rules.am.AccessMethodJobGenParams;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IntroduceMaterializationForInsertWithSelfScanRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.INSERT_DELETE) {
+ return false;
+ }
+
+ InsertDeleteOperator insertOp = (InsertDeleteOperator) op;
+ boolean sameDataset = checkIfInsertAndScanDatasetsSame(op, ((AqlDataSource) insertOp.getDataSource())
+ .getDataset().getDatasetName());
+
+ if (sameDataset) {
+ MaterializeOperator materializeOperator = new MaterializeOperator();
+ MaterializePOperator materializePOperator = new MaterializePOperator(true);
+ materializeOperator.setPhysicalOperator(materializePOperator);
+
+ ExtensionOperator extensionOperator = new ExtensionOperator(materializeOperator);
+ extensionOperator.setPhysicalOperator(materializePOperator);
+
+ extensionOperator.getInputs().add(
+ new MutableObject<ILogicalOperator>(insertOp.getInputs().get(0).getValue()));
+ context.computeAndSetTypeEnvironmentForOperator(extensionOperator);
+
+ insertOp.getInputs().clear();
+ insertOp.getInputs().add(new MutableObject<ILogicalOperator>(extensionOperator));
+ context.computeAndSetTypeEnvironmentForOperator(insertOp);
+ return true;
+ } else {
+ return false;
+ }
+
+ }
+
+ private boolean checkIfInsertAndScanDatasetsSame(AbstractLogicalOperator op, String insertDatasetName) {
+ boolean sameDataset = false;
+ for (int i = 0; i < op.getInputs().size(); ++i) {
+ AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+
+ if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+ UnnestMapOperator unnestMapOp = (UnnestMapOperator) descendantOp;
+ ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
+ if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+ FunctionIdentifier fid = f.getFunctionIdentifier();
+ if (!fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
+ throw new IllegalStateException();
+ }
+ AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+ jobGenParams.readFromFuncArgs(f.getArguments());
+ boolean isPrimaryIndex = jobGenParams.isPrimaryIndex();
+ String indexName = jobGenParams.getIndexName();
+ if (isPrimaryIndex && indexName.compareTo(insertDatasetName) == 0) {
+ return true;
+ }
+ }
+ } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
+ AqlDataSource ds = (AqlDataSource) dataSourceScanOp.getDataSource();
+ if (ds.getDatasourceType() != AqlDataSourceType.FEED
+ && ds.getDatasourceType() != AqlDataSourceType.EXTERNAL_FEED) {
+ if (ds.getDataset().getDatasetName().compareTo(insertDatasetName) == 0) {
+ return true;
+ }
+ }
+ }
+ sameDataset = checkIfInsertAndScanDatasetsSame(descendantOp, insertDatasetName);
+ if (sameDataset) {
+ break;
+ }
+ }
+ return sameDataset;
+ }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectAssignRule.java
similarity index 67%
rename from asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectRule.java
rename to asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectAssignRule.java
index d3e11ed2..98f7eeb 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRapidFrameFlushProjectAssignRule.java
@@ -22,21 +22,23 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
/**
- * This rule will search for project operators in an insert/delete/update plan and
- * pass a hint to all those projects between the first "insert" and the commit
- * operator. This hint is used by the project operator so that frames are pushed to
+ * This rule will search for project and assign operators in an insert/delete/update plan and
+ * pass a hint to all of them. This hint is used by the project and assign operators so that frames are pushed to
* the next operator without waiting until they get full. The purpose of this is to
- * reduce the time of holding exclusive locks on the keys that have been inserted.
+ * reduce the time of holding exclusive locks on the keys that have been inserted. Also to allow feeds batching
+ * to work correctly.
*
* @author salsubaiee
*/
-public class IntroduceRapidFrameFlushProjectRule implements IAlgebraicRewriteRule {
+public class IntroduceRapidFrameFlushProjectAssignRule implements IAlgebraicRewriteRule {
@Override
public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
@@ -52,44 +54,49 @@
return false;
}
- AbstractLogicalOperator descendantOp = op;
- while (descendantOp != null) {
- if (descendantOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ for (int i = 0; i < op.getInputs().size(); ++i) {
+ AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+
+ if (descendantOp.getOperatorTag() == LogicalOperatorTag.PROJECT
+ || descendantOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
if (descendantOp.getPhysicalOperator() == null) {
return false;
}
- } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE) {
- break;
}
- descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
+ checkIfRuleIsApplicable(descendantOp);
}
return true;
}
+ private boolean changeRule(AbstractLogicalOperator op) {
+ boolean planModified = false;
+ for (int i = 0; i < op.getInputs().size(); ++i) {
+ AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+
+ if (descendantOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ ProjectOperator projectOp = (ProjectOperator) descendantOp;
+ StreamProjectPOperator physicalOp = (StreamProjectPOperator) projectOp.getPhysicalOperator();
+ physicalOp.setRapidFrameFlush(true);
+ planModified = true;
+ } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator assignOp = (AssignOperator) descendantOp;
+ AssignPOperator physicalOp = (AssignPOperator) assignOp.getPhysicalOperator();
+ physicalOp.setRapidFrameFlush(true);
+ planModified = true;
+ }
+ changeRule(descendantOp);
+ }
+ return planModified;
+ }
+
@Override
public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
throws AlgebricksException {
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-
if (!checkIfRuleIsApplicable(op)) {
return false;
}
- AbstractLogicalOperator descendantOp = op;
- ProjectOperator projectOp = null;
-
- boolean planModified = false;
- while (descendantOp != null) {
- if (descendantOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
- projectOp = (ProjectOperator) descendantOp;
- StreamProjectPOperator physicalOp = (StreamProjectPOperator) projectOp.getPhysicalOperator();
- physicalOp.setRapidFrameFlush(true);
- planModified = true;
- } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE) {
- break;
- }
- descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
- }
- return planModified;
+ return changeRule(op);
}
}
\ No newline at end of file
diff --git a/asterix-app/data/csv/sample_01.csv b/asterix-app/data/csv/sample_01.csv
new file mode 100644
index 0000000..4dd437a
--- /dev/null
+++ b/asterix-app/data/csv/sample_01.csv
@@ -0,0 +1,8 @@
+1,0.899682764,5.6256,2013-08-07,07:22:35,1979-02-25T23:48:27.034
+2,0.669052398,,-1923-03-29,19:33:34,-1979-02-25T23:48:27.002
+3,0.572733058,192674,-1923-03-28,19:33:34,-1979-02-25T23:48:27.001
+4,,192674,-1923-03-27,19:33:34,-1979-02-25T23:48:27.001
+5,0.572733058,192674,,19:33:34,-1979-02-25T23:48:27.001
+6,0.572733058,192674,-1923-03-25,,-1979-02-25T23:48:27.001
+7,0.572733058,192674,-1923-03-24,19:33:34,
+8,,,,,
\ No newline at end of file
diff --git a/asterix-app/data/odd-numbers-2.adm b/asterix-app/data/odd-numbers-2.adm
new file mode 100644
index 0000000..c079eeb
--- /dev/null
+++ b/asterix-app/data/odd-numbers-2.adm
@@ -0,0 +1,5 @@
+{"id":1}
+{"id":3}
+{"id":5}
+{"id":7}
+{"id":9}
\ No newline at end of file
diff --git a/asterix-app/data/odd-numbers.adm b/asterix-app/data/odd-numbers.adm
new file mode 100644
index 0000000..8f400b6
--- /dev/null
+++ b/asterix-app/data/odd-numbers.adm
@@ -0,0 +1,10 @@
+{"id":1}
+{"id":3}
+{"id":5}
+{"id":7}
+{"id":9}
+{"id":11}
+{"id":13}
+{"id":15}
+{"id":17}
+{"id":19}
\ No newline at end of file
diff --git a/asterix-app/data/tpch0.001/partsupp.tbl b/asterix-app/data/tpch0.001/partsupp.tbl
index d8e5856..d7a631f 100644
--- a/asterix-app/data/tpch0.001/partsupp.tbl
+++ b/asterix-app/data/tpch0.001/partsupp.tbl
@@ -120,44 +120,44 @@
30|3|7945|583.84| sleep. bold, regular deposits hang doggedly furiously bold requests. slyly bold excuses detect busily above the even gifts. blithely express courts are carefully. blithely final packages until th|
31|2|9685|620.84|he blithely regular ideas. blithely unusual requests haggle fluffily. platelets|
31|7|1951|120.99|refully regular pinto beans. ironic requests integrate furiously since the quickly ruthless platelets. quickly ironic attainments ha|
-31|2|1402|761.64|r platelets nag blithely regular deposits. ironic, bold requests |
-31|7|137|849.11|blithely ironic accounts. slyly ironic asymptotes sleep ironic, even accounts. regular accounts thrash quickly|
+31|8|1402|761.64|r platelets nag blithely regular deposits. ironic, bold requests |
+31|10|137|849.11|blithely ironic accounts. slyly ironic asymptotes sleep ironic, even accounts. regular accounts thrash quickly|
32|3|2203|406.03|es? slyly enticing dugouts haggle carefully. regular packages alongside of the asymptotes are carefull|
32|8|467|109.34|ainst the unusual braids nod fluffily packages. regular packages nod among the slyly express|
-32|3|7975|747.14|final foxes boost furiously pending packages. quickly regular depths promise blithely accoun|
-32|8|7938|856.09|s integrate according to the even dependencies. carefully regular reque|
+32|9|7975|747.14|final foxes boost furiously pending packages. quickly regular depths promise blithely accoun|
+32|10|7938|856.09|s integrate according to the even dependencies. carefully regular reque|
33|4|4028|891.46|, pending requests affix slyly. slyly ironic deposits wake accounts. express accounts sleep slowly. ironic, express accounts run carefully fluffily final dependencies. furiously unusual ideas|
-33|9|4410|929.05| packages sleep carefully. slyly final instructions boost. slyly even requests among the carefully pending platelets wake along the final accounts. quickly expre|
-33|4|1287|310.76|dolites above the slyly express deposits try to haggle blithely special gifts. blithely ironic reque|
-33|9|6006|327.19|ly. ironic dependencies haggle carefully silent instructions. furiously ironic dolphins are fluffily furiously even theo|
+33|8|4410|929.05| packages sleep carefully. slyly final instructions boost. slyly even requests among the carefully pending platelets wake along the final accounts. quickly expre|
+33|9|1287|310.76|dolites above the slyly express deposits try to haggle blithely special gifts. blithely ironic reque|
+33|10|6006|327.19|ly. ironic dependencies haggle carefully silent instructions. furiously ironic dolphins are fluffily furiously even theo|
34|5|9934|848.75|ven instructions besides the gifts are furiously among the slyly regular packages! instructions use carefully. even requests sleep quickl|
-34|10|4749|265.31|ckly regular theodolites eat above the bravely regular courts. ironic requests wake slyly.|
-34|5|5459|824.69|ong the slyly silent requests. express, even requests haggle slyly|
+34|6|4749|265.31|ckly regular theodolites eat above the bravely regular courts. ironic requests wake slyly.|
+34|7|5459|824.69|ong the slyly silent requests. express, even requests haggle slyly|
34|10|5884|609.69|ully final tithes. slyly ironic deposits hang furiously about the regular, regular deposits|
35|6|2500|451.58|nic packages boost carefully carefully even theodolites. blithely fina|
35|1|8875|537.72|ully regular deposits: special accounts use. slyly final deposits wake slyly unusual, special ideas. asymptotes |
-35|6|596|669.19|slyly against the daring, pending accounts. fluffily special pinto beans integrate slyly after the carefully unusual packages. slyly bold accounts besides|
-35|1|2025|411.17|s cajole fluffily final deposits. furiously express packages after the blithely special realms boost evenly even requests. slow requests use above the unusual accoun|
-36|7|3907|630.91|al deposits detect fluffily fluffily unusual sauternes. carefully regular requests against the car|
-36|2|174|434.47|permanently express instructions. unusual accounts nag toward the accou|
-36|7|2625|569.91|ctions. pending requests are fluffily across the furiously regular notornis. unusu|
-36|2|8209|289.15|arefully regular requests cajole. special, express foxes sleep slowly. quickly unusual in|
+35|3|596|669.19|slyly against the daring, pending accounts. fluffily special pinto beans integrate slyly after the carefully unusual packages. slyly bold accounts besides|
+35|7|2025|411.17|s cajole fluffily final deposits. furiously express packages after the blithely special realms boost evenly even requests. slow requests use above the unusual accoun|
+36|8|3907|630.91|al deposits detect fluffily fluffily unusual sauternes. carefully regular requests against the car|
+36|9|174|434.47|permanently express instructions. unusual accounts nag toward the accou|
+36|10|2625|569.91|ctions. pending requests are fluffily across the furiously regular notornis. unusu|
+36|11|8209|289.15|arefully regular requests cajole. special, express foxes sleep slowly. quickly unusual in|
37|8|7171|824.96|usly into the slyly final requests. ironic accounts are furiously furiously ironic i|
37|3|5542|126.59|ven deposits. ironic foxes cajole. slyly final deposits are furiously after the furiously even packages. slyly ironic platelets toward the slyl|
-37|8|7113|15.72|re bravely along the furiously express requests. blithely special asymptotes are quickly. fluffily regular packages alo|
-37|3|1449|745.64|y after the ironic accounts. blithely final instructions affix blithely. bold packages sleep carefully regular instructions. regular packages affix carefully. stealthy fo|
+37|7|7113|15.72|re bravely along the furiously express requests. blithely special asymptotes are quickly. fluffily regular packages alo|
+37|9|1449|745.64|y after the ironic accounts. blithely final instructions affix blithely. bold packages sleep carefully regular instructions. regular packages affix carefully. stealthy fo|
38|9|1226|570.11| slyly even pinto beans. blithely special requests nag slyly about the ironic packages. |
38|4|4237|662.75|lar warhorses cajole evenly against the attainments. requests cajole furiously furiously express requests. carefully regular platelets use fluffily after the silent, unusual ideas: bl|
-38|9|1135|160.70|express accounts haggle. carefully even pinto beans according to the slyly final foxes nag slyly about the enticingly express dol|
-38|4|3516|847.09|nal accounts. furiously pending hockey players solve slyly after the furiously final dependencies. deposits are blithely. carefully regular packages unwind busily at the deposits. fluffily |
+38|7|1135|160.70|express accounts haggle. carefully even pinto beans according to the slyly final foxes nag slyly about the enticingly express dol|
+38|8|3516|847.09|nal accounts. furiously pending hockey players solve slyly after the furiously final dependencies. deposits are blithely. carefully regular packages unwind busily at the deposits. fluffily |
39|10|3633|463.10|kages are slyly above the slyly pending pinto beans. bold, ironic pinto beans sleep against the blithely regular requests. fluffily even pinto beans use. regular theodolites haggle against the quic|
39|5|3682|300.43|ng requests are according to the packages. regular packages boost quickly. express Tiresias sleep silently across the even, regular ideas! blithely iro|
-39|10|5475|532.26| beans cajole carefully carefully express requests. instructions sleep furiously bold deposits. furiously regular depos|
-39|5|6259|737.86|y. special, even asymptotes cajole carefully ironic accounts. regular, final pinto beans cajole quickly. regular requests use warhorses. special, special accounts hinder boldly across the|
+39|6|5475|532.26| beans cajole carefully carefully express requests. instructions sleep furiously bold deposits. furiously regular depos|
+39|3|6259|737.86|y. special, even asymptotes cajole carefully ironic accounts. regular, final pinto beans cajole quickly. regular requests use warhorses. special, special accounts hinder boldly across the|
40|1|7690|776.13|lets use fluffily carefully final deposits. blithely ironic instructions sublate against the furiously final ideas; slyly bold courts x-ray silent foxes. regular foxes wake blithely. slyl|
40|6|1704|565.82|riously furiously silent asymptotes. final deposits cajole blithely ironic requests. furiously special pains into the blithely final instru|
-40|1|4521|374.71|ptotes haggle. slyly even requests nag fluffily silent packages. blith|
-40|6|6617|196.64|he slyly unusual epitaphs? ironic deposits at the furiously unusual instructions thrash blithely requests. requests are carefully blithely pending waters.|
+40|7|4521|374.71|ptotes haggle. slyly even requests nag fluffily silent packages. blith|
+40|8|6617|196.64|he slyly unusual epitaphs? ironic deposits at the furiously unusual instructions thrash blithely requests. requests are carefully blithely pending waters.|
41|2|9040|488.55|ss the dinos wake along the blithely regular theodolites. foxes cajole quickly ironic, final foxes. blithely ironic packages haggle against |
41|8|5946|391.81| slyly slyly regular requests. final deposits sleep fluffily. blithely bold instructions detect carefully. blithely pending requests are furiously ironically final ideas. regul|
41|4|1550|916.55| the blithely final ideas. furiously regular asymptotes could cajole furious|
@@ -319,45 +319,45 @@
80|9|5385|945.72|cial asymptotes believe after the blithely unusual deposits. furiously silent pinto beans cajole quickly inside the slyly even deposits. regular, f|
80|8|4034|797.05|ptotes cajole carefully. express ideas cajole carefully even somas. final pinto beans print fluffily across the |
81|2|1605|550.29|es haggle blithely fluffily final requests. furiously regular foxes use. furiously unusual requests outside the furiously regular requests|
-81|2|5923|220.23|the final, quick accounts are blithely above the s|
-81|2|2942|409.73|accounts boost. fluffily unusual requests cajole fluffily slyly ironic requests. foxes cajole quick|
-81|2|58|492.19| instructions boost furiously across the foxes-- final depo|
+81|3|5923|220.23|the final, quick accounts are blithely above the s|
+81|4|2942|409.73|accounts boost. fluffily unusual requests cajole fluffily slyly ironic requests. foxes cajole quick|
+81|5|58|492.19| instructions boost furiously across the foxes-- final depo|
82|3|7793|697.31|he accounts cajole quickly after the even patterns. ironic platelets sublate regular, even asymptotes. quick courts affix according to|
-82|3|7698|585.86|pinto beans. slyly express excuses haggle. blithely even pinto beans about the quick inst|
-82|3|8268|604.25|e after the carefully even theodolites. regular, pending accounts boost. quickly final asymptotes haggle slyly. requests use final, bold pinto beans. bold, ruthle|
-82|3|5532|900.07| slyly? fluffily special dependencies haggle among the slyly special requests. regular, bold packages after the blithely ironic packages are slyly ironic packages. slyly final deposits w|
+82|4|7698|585.86|pinto beans. slyly express excuses haggle. blithely even pinto beans about the quick inst|
+82|5|8268|604.25|e after the carefully even theodolites. regular, pending accounts boost. quickly final asymptotes haggle slyly. requests use final, bold pinto beans. bold, ruthle|
+82|6|5532|900.07| slyly? fluffily special dependencies haggle among the slyly special requests. regular, bold packages after the blithely ironic packages are slyly ironic packages. slyly final deposits w|
83|4|3010|745.51|l foxes along the bold, regular packages integrate carefully express courts! final excuses sleep carefully ironic|
-83|4|8200|399.64|y final platelets are carefully carefully special platelets. carefully ironic requests wake blithely alongside of the slyly even accounts. bold, regular requests sleep |
-83|4|5974|657.22| even packages boost furiously. slyly regular gifts above the accounts are quickly express packages. slyly pending deposits besides the express, even asymptotes haggle after the ironic ins|
-83|4|3890|24.73|deposits. carefully even dependencies across the dependencies haggl|
+83|5|8200|399.64|y final platelets are carefully carefully special platelets. carefully ironic requests wake blithely alongside of the slyly even accounts. bold, regular requests sleep |
+83|6|5974|657.22| even packages boost furiously. slyly regular gifts above the accounts are quickly express packages. slyly pending deposits besides the express, even asymptotes haggle after the ironic ins|
+83|7|3890|24.73|deposits. carefully even dependencies across the dependencies haggl|
84|5|5711|233.61|arefully final platelets cajole blithely; quickly final accounts use furiously. furiously reg|
-84|5|208|469.80|carefully express dolphins nag about the slyly bold requests. slyly even packages wake among the furiously special attainments.|
-84|5|2909|969.44|silent requests cajole slowly bold ideas. special, special deposits according to the always silent packages are against the furiously silent packages. even, blithe accounts sleep slyly across |
-84|5|903|707.77|gly regular dependencies boost. slyly even accounts sleep. furiously final hockey players wake carefully with the reg|
+84|6|208|469.80|carefully express dolphins nag about the slyly bold requests. slyly even packages wake among the furiously special attainments.|
+84|7|2909|969.44|silent requests cajole slowly bold ideas. special, special deposits according to the always silent packages are against the furiously silent packages. even, blithe accounts sleep slyly across |
+84|8|903|707.77|gly regular dependencies boost. slyly even accounts sleep. furiously final hockey players wake carefully with the reg|
85|6|2628|608.77|xes wake furiously after the carefully even platelets. blithe theodolites are furi|
-85|6|118|917.83| against the even deposits. furiously bold ideas along the furious requ|
-85|6|2074|491.20|encies-- slyly regular requests about the quiet accounts detect quickly at the |
-85|6|8289|73.81|s cajole slyly along the slyly special accounts. regular, special deposits wake. furiously special foxes boost. blithely even packa|
+85|5|118|917.83| against the even deposits. furiously bold ideas along the furious requ|
+85|4|2074|491.20|encies-- slyly regular requests about the quiet accounts detect quickly at the |
+85|3|8289|73.81|s cajole slyly along the slyly special accounts. regular, special deposits wake. furiously special foxes boost. blithely even packa|
86|7|806|65.98|ackages. blithely pending accounts are slyly furiously pending theodolites. furiously eve|
-86|7|2773|250.04|ding accounts. slyly special requests will have to affix carefully along the furiously unusual packages. regular theodol|
-86|7|5546|816.53|s. slyly final requests wake. furious deposits must wake blithely among the blithely ironic instructions. special hockey players try to are bli|
-86|7|1418|332.65|press theodolites sleep carefully about the blithely unusual requests. quickly final deposits breach slyly |
+86|1|2773|250.04|ding accounts. slyly special requests will have to affix carefully along the furiously unusual packages. regular theodol|
+86|2|5546|816.53|s. slyly final requests wake. furious deposits must wake blithely among the blithely ironic instructions. special hockey players try to are bli|
+86|3|1418|332.65|press theodolites sleep carefully about the blithely unusual requests. quickly final deposits breach slyly |
87|8|5679|688.33|t the carefully regular asymptotes. blithely stealthy pinto beans within the furiously expres|
-87|8|1272|435.42|ronic foxes sleep along the special foxes. final ideas wake quickly about the carefully special theodolites. blithely ironic packages are blithely. regular, regular pint|
-87|8|9041|617.20|furiously final deposits. furiously special dependencies solve across the regular, special ideas. carefully silent requests haggle furiously after the special, specia|
-87|8|1892|868.60|arhorses are. unusual requests use blithely furiously final ideas. final requests sleep theodoli|
+87|2|1272|435.42|ronic foxes sleep along the special foxes. final ideas wake quickly about the carefully special theodolites. blithely ironic packages are blithely. regular, regular pint|
+87|3|9041|617.20|furiously final deposits. furiously special dependencies solve across the regular, special ideas. carefully silent requests haggle furiously after the special, specia|
+87|4|1892|868.60|arhorses are. unusual requests use blithely furiously final ideas. final requests sleep theodoli|
88|9|6116|334.58|ect furiously around the regular deposits. special, final platelets boost furiously. blithely unusu|
-88|9|395|71.50| the regular accounts-- furiously even accounts use quickly after the regular, regular deposits. furiously e|
-88|9|9979|81.82|f the regular, regular requests believe fluffily along the final, quiet decoys. furiously even accounts cajole. carefully express requests wake quickly among the ideas. quickly silent |
-88|9|276|821.43|gular pinto beans. slyly pending excuses breach blithely express accounts. thin deposits sleep slyly around the even accounts; fluffily busy patterns kindle. slyly final deposits along the |
+88|1|395|71.50| the regular accounts-- furiously even accounts use quickly after the regular, regular deposits. furiously e|
+88|2|9979|81.82|f the regular, regular requests believe fluffily along the final, quiet decoys. furiously even accounts cajole. carefully express requests wake quickly among the ideas. quickly silent |
+88|3|276|821.43|gular pinto beans. slyly pending excuses breach blithely express accounts. thin deposits sleep slyly around the even accounts; fluffily busy patterns kindle. slyly final deposits along the |
89|10|3430|744.87| integrate slyly dolphins. bold, final frets use beside the carefully even accounts. slyly close dependencies sleep quickly carefully final pinto beans. foxes promi|
-89|10|8599|776.53|ress packages use furiously. furiously regular packages thrash blithely about the slyly pe|
-89|10|7876|417.61|nstructions: furiously even requests are quietly unusual accounts. regular requests are after the blithely regular deposits. sl|
-89|10|924|920.02|ickly unusual asymptotes after the slyly unusual accounts are carefully doggedly ironic accounts. even, final accounts use furiousl|
+89|1|8599|776.53|ress packages use furiously. furiously regular packages thrash blithely about the slyly pe|
+89|2|7876|417.61|nstructions: furiously even requests are quietly unusual accounts. regular requests are after the blithely regular deposits. sl|
+89|3|924|920.02|ickly unusual asymptotes after the slyly unusual accounts are carefully doggedly ironic accounts. even, final accounts use furiousl|
90|1|8037|409.38|eas. unusual, pending packages boost quietly final accounts. slyly final packages serve. slyly even instructions sleep carefully. quickly even foxes wake quickly. |
-90|1|9683|498.43| accounts! fluffily regular deposits x-ray about the unusual, final packages. furiously final deposits alongside of the caref|
-90|1|7849|666.13|carefully ironic accounts are around the slyly bold asymptotes. carefully regular packages use furiously. ironic platelets affix carefully final accounts-- fluffily final pinto beans across the fina|
-90|1|7629|50.84|onic requests wake fluffily unusual packages. furiously even frays after the daringly pending requests wake furiously alongside of the bold requests. fluffily ironic ideas nag. ironic,|
+90|2|9683|498.43| accounts! fluffily regular deposits x-ray about the unusual, final packages. furiously final deposits alongside of the caref|
+90|3|7849|666.13|carefully ironic accounts are around the slyly bold asymptotes. carefully regular packages use furiously. ironic platelets affix carefully final accounts-- fluffily final pinto beans across the fina|
+90|4|7629|50.84|onic requests wake fluffily unusual packages. furiously even frays after the daringly pending requests wake furiously alongside of the bold requests. fluffily ironic ideas nag. ironic,|
91|2|7986|528.64|luffily final instructions. furiously unusual foxes haggle |
91|3|3257|906.20|ackages cajole slyly. blithely bold deposits cajole. blithely |
91|4|483|823.21|n: slyly ironic foxes nag blithely according to the furiously bold foxes. regular, regular accounts a|
@@ -520,44 +520,44 @@
130|3|7387|883.99|aggle furiously. even ideas hinder deposits. even, final ideas are. unusual theodolites after the special, express foxes haggle carefully pending accou|
131|2|3263|211.70|sits sleep quickly regular multipliers. slyly even platelets cajole after the furiously ironic deposits. slyly ironic requests should have to cajole: bl|
131|7|125|861.84|l accounts grow quickly-- slyly ironic requests haggle? quickly express pinto bean|
-131|2|5138|572.43|grouches run with the carefully even packages. ironic, even deposits run slyly along the packages. special dependencies among the regular |
-131|7|8945|613.09| are carefully along the quickly final theodolites. packages after the quickly pending package|
+131|3|5138|572.43|grouches run with the carefully even packages. ironic, even deposits run slyly along the packages. special dependencies among the regular |
+131|4|8945|613.09| are carefully along the quickly final theodolites. packages after the quickly pending package|
132|3|3092|687.29|y special decoys against the ideas affix against the sly|
132|8|1904|925.73|the regular foxes wake ironic deposits. ironic, special requests use blithely instructions! final requests hang. blithely regular deposits haggle. ir|
-132|3|7441|357.06|ests. furiously unusual requests wake furiously. quickly unusual depos|
-132|8|5303|353.06|ep blithely after the sly accounts. slyly express dolphins cajole amon|
+132|4|7441|357.06|ests. furiously unusual requests wake furiously. quickly unusual depos|
+132|5|5303|353.06|ep blithely after the sly accounts. slyly express dolphins cajole amon|
133|4|5727|49.17|boost blithely across the ironic, regular instructions. packages use slyly unusual requests. bold accounts above the fu|
133|9|404|478.18|ly ironic requests run instead of the blithely ironic accounts? regular ideas use fluffily: even, express packages sleep abov|
-133|4|4568|57.48|dolites. ironic accounts are blithely pinto beans. regular pinto beans haggle beneath|
-133|9|2813|277.26|s. pending, final accounts haggle blithely furiously pending deposits! carefully unusual attainments integrate. blithely bo|
+133|5|4568|57.48|dolites. ironic accounts are blithely pinto beans. regular pinto beans haggle beneath|
+133|6|2813|277.26|s. pending, final accounts haggle blithely furiously pending deposits! carefully unusual attainments integrate. blithely bo|
134|5|8879|848.14|lites. slyly final foxes after the bold requests cajole carefu|
134|10|9013|102.99|pendencies. furiously express warthogs cajole furiously ironic, regular asymptotes. bold deposits boost among the furiously even theodolites. regular instructions integrate carefully |
-134|5|852|927.45| sleep unusual, express packages. unusual sentiments are furio|
-134|10|6270|388.28| to the furiously pending deposits nag along the slyly express asymptotes. slyly silent accounts shal|
+134|6|852|927.45| sleep unusual, express packages. unusual sentiments are furio|
+134|7|6270|388.28| to the furiously pending deposits nag along the slyly express asymptotes. slyly silent accounts shal|
135|6|6940|465.82|ding foxes cajole. even dugouts haggle busily. fluffily pending packages about the express excuses boost slyly final packages. blithely express ideas cajole about the carefu|
135|1|2443|9.83|atterns. pending, special deposits are furiously. express, regular deposits integrate quickly. unusual gifts cajole blithely stealthily pending deposit|
-135|6|7453|698.42|ven accounts. slyly final instructions nag slyly around the regular, unusual packages. slyly sp|
-135|1|2771|306.43|old deposits. furiously express instructions boost. pending dolphins use requests. slyly regular packages cajole quickly final ideas. pending, regular ideas nag carefully even, express pla|
+135|2|7453|698.42|ven accounts. slyly final instructions nag slyly around the regular, unusual packages. slyly sp|
+135|3|2771|306.43|old deposits. furiously express instructions boost. pending dolphins use requests. slyly regular packages cajole quickly final ideas. pending, regular ideas nag carefully even, express pla|
136|7|2237|548.19|ond the silent accounts haggle above the blithely regular packages|
136|2|6068|806.19|structions. ironic theodolites haggle according to the final, daring pearls. carefully ironic somas are silently requests. express pa|
-136|7|8979|387.57|ans. express pinto beans wake carefully among the slyly ironic foxes: carefully final pinto beans haggle blithely. pending, final deposits promise furiously|
-136|2|9617|525.81| across the carefully pending warthogs. close, regular packages are quickly after the never ironic foxes. accounts sleep quickly along the furiously regular re|
+136|1|8979|387.57|ans. express pinto beans wake carefully among the slyly ironic foxes: carefully final pinto beans haggle blithely. pending, final deposits promise furiously|
+136|3|9617|525.81| across the carefully pending warthogs. close, regular packages are quickly after the never ironic foxes. accounts sleep quickly along the furiously regular re|
137|8|9057|302.26|slyly about the regular instructions. even, ironic theodolites use carefully around the even decoys. unusual, pending dolphin|
137|3|4078|441.11|packages. blithely unusual sentiments should are. furiously regular accounts nag quickly carefully special asymptotes! idly ironic requests dazzle bold requests. carefully expres|
-137|8|467|371.85|ly special accounts detect carefully. furiously ironic deposits nag express packages. slyly quiet |
-137|3|7850|187.31|atelets sublate fluffily. enticingly unusual packages boost according to the blithely ironic foxes. pending requests mold sly|
+137|2|467|371.85|ly special accounts detect carefully. furiously ironic deposits nag express packages. slyly quiet |
+137|4|7850|187.31|atelets sublate fluffily. enticingly unusual packages boost according to the blithely ironic foxes. pending requests mold sly|
138|9|133|576.96|regular, final deposits maintain slyly even requests. regularly furious deposits use above the stealthy requests. ironic deposits are. carefully final frays are carefully. carefu|
138|4|2535|885.35|lar deposits. courts sleep carefully. furiously express ideas boost furiously after the final, regular foxes. furiously bold deposits are. express accounts haggle blithely. |
-138|9|7907|119.83|epitaphs? quickly express foxes use pending accounts. special packages cajole blithely among the quickly unusual accounts? boldly ironic packages across the slyly ironic senti|
-138|4|967|309.03|pendencies integrate against the unusual pains. carefully unusual theodolites wake quickly across the deposits. blithely regular deposits alongside of the carefully regular deposits|
+138|1|7907|119.83|epitaphs? quickly express foxes use pending accounts. special packages cajole blithely among the quickly unusual accounts? boldly ironic packages across the slyly ironic senti|
+138|2|967|309.03|pendencies integrate against the unusual pains. carefully unusual theodolites wake quickly across the deposits. blithely regular deposits alongside of the carefully regular deposits|
139|10|2886|285.75|fully ironic requests according to the quickly final idea|
139|5|9255|684.61|ickly furiously regular excuses. boldly express deposits sleep. ideas nag above the silent dependencies. slyly regular packages wake furiously. requests are carefully. quickly final fox|
-139|10|1042|972.23|gular, regular theodolites. regular asymptotes haggle carefully according to the permanently even deposits. slyly special account|
-139|5|3285|690.00|xpress pains. quickly regular ideas after the special, bold excuses wake furiously final ideas. slyly bold accounts nag packages. ironically regular|
+139|1|1042|972.23|gular, regular theodolites. regular asymptotes haggle carefully according to the permanently even deposits. slyly special account|
+139|3|3285|690.00|xpress pains. quickly regular ideas after the special, bold excuses wake furiously final ideas. slyly bold accounts nag packages. ironically regular|
140|1|2379|501.05|of the silent, bold courts. slyly regular dependencies haggle. fluffily special deposits cajole carefully. quickly ironic depos|
140|6|3533|781.45|ayers. carefully ironic pinto beans nod carefully furiously regular pinto beans. slyly ironic requests after the carefully regular packages are about the blithel|
-140|1|304|45.84|ing requests. carefully unusual foxes are final requests. slyly regular accounts wake permanently. quickly ironic theodolites hagg|
-140|6|7346|429.52| special pinto beans wake carefully unusual warthogs! furi|
+140|2|304|45.84|ing requests. carefully unusual foxes are final requests. slyly regular accounts wake permanently. quickly ironic theodolites hagg|
+140|3|7346|429.52| special pinto beans wake carefully unusual warthogs! furi|
141|2|6776|293.63|fluffily unusual courts sleep. close pinto beans haggle quickly after the carefully ir|
141|8|1660|139.18|egular accounts. enticingly bold theodolites eat slyly across the never ironic platelets. theodolites wake bli|
141|4|7628|838.08|sly about the pinto beans. blithely ironic ideas sleep. foxes are quietly among the pinto beans. carefu|
@@ -719,45 +719,45 @@
180|9|724|426.16|e, regular accounts. furiously final ideas are furiously above the bold, silent asymptotes. sly instructions are carefully quickly final sentiments. furiously ironic foxes cajole bold, exp|
180|8|5899|864.83|hin the carefully furious pinto beans. furiously ironic pinto beans use slyly above the even instructio|
181|2|2416|844.44|ully. theodolites throughout the blithely unusual pinto bea|
-181|2|3242|886.53| express ideas nag carefully brave accounts. slyly express deposits would affix. final, special requests against the slyl|
-181|2|215|938.29| accounts boost furiously furiously blithe theodolites. slyly bold requests unwind special, unusual requests. furious ideas boost quickly pending |
-181|2|1122|657.25|lyly fluffily pending foxes. fluffily ironic pains haggle. thinly regular requests against the deposits affix after the never ev|
+181|3|3242|886.53| express ideas nag carefully brave accounts. slyly express deposits would affix. final, special requests against the slyl|
+181|4|215|938.29| accounts boost furiously furiously blithe theodolites. slyly bold requests unwind special, unusual requests. furious ideas boost quickly pending |
+181|5|1122|657.25|lyly fluffily pending foxes. fluffily ironic pains haggle. thinly regular requests against the deposits affix after the never ev|
182|3|9699|535.27|ound the furiously regular foxes. pending requests dazzle along |
-182|3|960|519.36|arefully pending dependencies are always slyly unusual pin|
-182|3|6243|741.46|accounts are slyly. furiously ironic requests haggle. express, special instructions against the ironic theodolites use s|
-182|3|6146|365.00|s. blithely express theodolites sleep blithely alongside of the requests?|
+182|4|960|519.36|arefully pending dependencies are always slyly unusual pin|
+182|5|6243|741.46|accounts are slyly. furiously ironic requests haggle. express, special instructions against the ironic theodolites use s|
+182|6|6146|365.00|s. blithely express theodolites sleep blithely alongside of the requests?|
183|4|30|875.44|slyly. furiously regular instructions cajole slyly about the pending, final theodolites. blithely final deposits cajole fluffily alo|
-183|4|4482|424.86|es. depths affix fluffily. bold instructions haggle. ruthless instructions must have to boost|
-183|4|8707|884.26|posits wake. blithely pending requests nag furiously alongside of the p|
-183|4|333|678.16|ost final, final theodolites. slyly bold foxes dazzle carefully furiously regular accounts. regular, sly instructions about the furiously regular excuses nag blithely abou|
+183|5|4482|424.86|es. depths affix fluffily. bold instructions haggle. ruthless instructions must have to boost|
+183|6|8707|884.26|posits wake. blithely pending requests nag furiously alongside of the p|
+183|7|333|678.16|ost final, final theodolites. slyly bold foxes dazzle carefully furiously regular accounts. regular, sly instructions about the furiously regular excuses nag blithely abou|
184|5|7069|449.45|nal ideas. blithely final ideas haggle against the pinto beans. qu|
-184|5|9193|576.88|uickly quick dependencies could detect furiously. final packages p|
-184|5|6400|551.90|ss dependencies. quickly even pinto beans are. express accounts a|
-184|5|831|186.84|kages cajole carefully furiously ironic instructions. deposits use bl|
+184|1|9193|576.88|uickly quick dependencies could detect furiously. final packages p|
+184|2|6400|551.90|ss dependencies. quickly even pinto beans are. express accounts a|
+184|3|831|186.84|kages cajole carefully furiously ironic instructions. deposits use bl|
185|6|1475|538.58|unts hinder slyly. quickly express ideas sleep carefully |
-185|6|6244|213.04|ly unusual decoys are furiously quickly regular packages. bold, ironic foxes cajole fluffily around|
-185|6|7245|426.74|sleep blithely alongside of the regular excuses. even, regular|
-185|6|8014|510.23|lithely even ideas. regular platelets wake carefully ironic, special instructions! final pearls above the fluffily quiet ideas use furiously about the |
+185|1|6244|213.04|ly unusual decoys are furiously quickly regular packages. bold, ironic foxes cajole fluffily around|
+185|2|7245|426.74|sleep blithely alongside of the regular excuses. even, regular|
+185|3|8014|510.23|lithely even ideas. regular platelets wake carefully ironic, special instructions! final pearls above the fluffily quiet ideas use furiously about the |
186|7|1095|252.84|. carefully regular pinto beans according to the blithely close asymptotes haggle carefully special requests. packages cajole up the furi|
-186|7|1945|18.75|nic foxes boost carefully careful packages: express, fluffy dolphins nag quickly ironic packages. slyly bold requests nag amon|
-186|7|8838|729.42|ing asymptotes. enticingly regular theodolites mai|
-186|7|7898|812.37|ctions sleep silently carefully bold platelets. furiously ironic dependencies boost. regular de|
+186|1|1945|18.75|nic foxes boost carefully careful packages: express, fluffy dolphins nag quickly ironic packages. slyly bold requests nag amon|
+186|2|8838|729.42|ing asymptotes. enticingly regular theodolites mai|
+186|3|7898|812.37|ctions sleep silently carefully bold platelets. furiously ironic dependencies boost. regular de|
187|8|8656|238.66|tes use along the even foxes? final foxes haggle pinto beans. slyly ironic theodolites are according to the deposits. furiously pending reques|
-187|8|4945|316.64|eposits boost quickly bold requests. furiously regular ideas boost boldly. special, express dependencies are fluffily slyly reg|
-187|8|3183|362.75|t the bold platelets. fluffily express platelets cajole fluffily along the always bold requests. blith|
-187|8|7440|989.71|e slyly against the slyly regular pinto beans. requests haggle carefully around the asymptotes. regular, regular asymptotes use furiously some|
+187|1|4945|316.64|eposits boost quickly bold requests. furiously regular ideas boost boldly. special, express dependencies are fluffily slyly reg|
+187|2|3183|362.75|t the bold platelets. fluffily express platelets cajole fluffily along the always bold requests. blith|
+187|3|7440|989.71|e slyly against the slyly regular pinto beans. requests haggle carefully around the asymptotes. regular, regular asymptotes use furiously some|
188|9|4835|771.95|pains are fluffily about the fluffily pending asymptot|
-188|9|2620|331.70|elets nag slyly regular pinto beans. slyly even dugouts above the blithely unusual theodolites su|
-188|9|730|713.62|nag against the final accounts. blithely pending attainments lose. silent requests wake quickly. careful|
-188|9|5430|920.20|uriously. special, regular instructions sleep along the accounts. quickly even foxes across the regular theodolites hang u|
+188|1|2620|331.70|elets nag slyly regular pinto beans. slyly even dugouts above the blithely unusual theodolites su|
+188|2|730|713.62|nag against the final accounts. blithely pending attainments lose. silent requests wake quickly. careful|
+188|3|5430|920.20|uriously. special, regular instructions sleep along the accounts. quickly even foxes across the regular theodolites hang u|
189|10|1305|392.50|packages. regular, unusual accounts lose furiously fluffily regular platelets. requests sleep carefully dependenc|
-189|10|8777|573.22|beans cajole slyly ironic requests. requests are quickly unusual, even packages. ironic frays haggle. blithely pending requests nod slyly. express, silent requests against the slyly unusual |
-189|10|6369|946.07|ts hinder slyly regular, unusual foxes. final sentiments use above the slyly r|
-189|10|2505|593.23| the deposits. special deposits sleep-- furiously regular sauternes solve furiously across the furiously regular pack|
+189|1|8777|573.22|beans cajole slyly ironic requests. requests are quickly unusual, even packages. ironic frays haggle. blithely pending requests nod slyly. express, silent requests against the slyly unusual |
+189|2|6369|946.07|ts hinder slyly regular, unusual foxes. final sentiments use above the slyly r|
+189|3|2505|593.23| the deposits. special deposits sleep-- furiously regular sauternes solve furiously across the furiously regular pack|
190|1|535|621.53|unts must have to haggle; slyly ironic accounts affix slyly alongside of the carefully even accounts. furious deposits haggle quietly among the packages. blithely |
-190|1|5845|608.91| haggle along the carefully unusual pinto beans. quickly final accounts sleep a|
-190|1|4579|396.60|inal, final foxes. regular, even deposits wake blithely! silent, regular packages integrate according to the slyly regular deposits. ironic, ironic notornis ha|
-190|1|2861|458.00|s cajole slyly across the daring, final pinto beans. carefully quiet requests affix along the a|
+190|2|5845|608.91| haggle along the carefully unusual pinto beans. quickly final accounts sleep a|
+190|3|4579|396.60|inal, final foxes. regular, even deposits wake blithely! silent, regular packages integrate according to the slyly regular deposits. ironic, ironic notornis ha|
+190|4|2861|458.00|s cajole slyly across the daring, final pinto beans. carefully quiet requests affix along the a|
191|2|8310|521.06|the slowly regular deposits. special accounts along the quickly unusual|
191|3|1693|464.46|y. slyly unusual waters across the special pinto beans nag blithely according to the busy deposits. carefully regular accounts are against the regular accounts; perman|
191|4|597|126.96|ly final accounts should have to boost above the doggedly express pinto beans. blithely regular packages cajole furiously bold requests. fluf|
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
index 9fcbb29..c10818f 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
@@ -15,6 +15,7 @@
package edu.uci.ics.asterix.api.common;
import java.io.IOException;
+import java.util.List;
import java.util.logging.Logger;
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
@@ -45,7 +46,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
import edu.uci.ics.hyracks.storage.common.buffercache.DelayPageCleanerPolicy;
@@ -118,7 +119,8 @@
storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages(),
storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory());
- lsmIOScheduler = SynchronousScheduler.INSTANCE;
+ AsynchronousScheduler.INSTANCE.init(ncApplicationContext.getThreadFactory());
+ lsmIOScheduler = AsynchronousScheduler.INSTANCE;
mergePolicy = new ConstantMergePolicy(storageProperties.getLSMIndexMergeThreshold(), this);
ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
@@ -223,8 +225,8 @@
}
@Override
- public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
- return indexLifecycleManager.getVirtualBufferCache(datasetID);
+ public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
+ return indexLifecycleManager.getVirtualBufferCaches(datasetID);
}
@Override
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
index 712d993..b528381 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
@@ -14,12 +14,12 @@
*/
package edu.uci.ics.asterix.api.common;
+import java.util.List;
+
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -88,32 +88,8 @@
}
@Override
- public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
- return asterixAppRuntimeContext.getVirtualBufferCache(datasetID);
- }
-
- @Override
- public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider() {
- return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
- }
-
- @Override
- public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider(boolean isPrimary) {
- if (isPrimary) {
- return AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER;
- } else {
- return AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER;
- }
- }
-
- @Override
- public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider() {
- return AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER;
- }
-
- @Override
- public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider() {
- return AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER;
+ public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
+ return asterixAppRuntimeContext.getVirtualBufferCaches(datasetID);
}
@Override
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index 94eefb6..3c7f455 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
@@ -145,6 +145,7 @@
ADDED_PENDINGOP_RECORD_TO_METADATA
}
+ public static final boolean IS_DEBUG_MODE = false;//true
private final List<Statement> aqlStatements;
private final PrintWriter out;
private final SessionConfig sessionConfig;
@@ -191,7 +192,6 @@
IAWriterFactory writerFactory = PrinterBasedWriterFactory.INSTANCE;
IResultSerializerFactoryProvider resultSerializerFactoryProvider = ResultSerializerFactoryProvider.INSTANCE;
Map<String, String> config = new HashMap<String, String>();
- List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
for (Statement stmt : aqlStatements) {
validateOperation(activeDefaultDataverse, stmt);
@@ -200,7 +200,6 @@
metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
metadataProvider.setOutputFile(outputFile);
metadataProvider.setConfig(config);
- jobsToExecute.clear();
switch (stmt.getKind()) {
case SET: {
handleSetStatement(metadataProvider, stmt, config);
@@ -1483,6 +1482,7 @@
try {
ConnectFeedStatement cfs = (ConnectFeedStatement) stmt;
String dataverseName = getActiveDataverseName(cfs.getDataverseName());
+ metadataProvider.setWriteTransaction(true);
CompiledConnectFeedStatement cbfs = new CompiledConnectFeedStatement(dataverseName, cfs.getFeedName(), cfs
.getDatasetName().getValue(), cfs.getPolicy(), cfs.getQuery(), cfs.getVarCounter());
@@ -1786,6 +1786,9 @@
private void abort(Exception rootE, Exception parentE, MetadataTransactionContext mdTxnCtx) {
try {
+ if (IS_DEBUG_MODE) {
+ rootE.printStackTrace();
+ }
MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
} catch (Exception e2) {
parentE.addSuppressed(e2);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index 7f41b4a..37c0e8d 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -30,6 +30,7 @@
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
@@ -123,12 +124,11 @@
datasetName);
AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate()));
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
splitsAndConstraint.second);
@@ -175,13 +175,12 @@
localResourceMetadata, LocalResource.LSMBTreeResource);
TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields,
new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new PrimaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
@@ -260,15 +259,15 @@
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad;
if (!loadStmt.alreadySorted()) {
btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, splitsAndConstraint.first, typeTraits,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, splitsAndConstraint.first, typeTraits,
comparatorFactories, blooFilterKeyFields, fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR,
true, numElementsHint, true, new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
splitsAndConstraint.second);
@@ -286,15 +285,15 @@
spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, btreeBulkLoad, 0);
} else {
btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, splitsAndConstraint.first, typeTraits,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, splitsAndConstraint.first, typeTraits,
comparatorFactories, blooFilterKeyFields, fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR,
- false, numElementsHint, true, new LSMBTreeDataflowHelperFactory(
+ true, numElementsHint, true, new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
splitsAndConstraint.second);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
index 7bd6c69..bf31db7 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
@@ -18,10 +18,12 @@
import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
@@ -66,12 +68,11 @@
.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, indexName);
AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate()));
AlgebricksPartitionConstraintHelper
.setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
index 052368a..3ae5f6d 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -19,6 +19,8 @@
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -60,13 +62,12 @@
localResourceMetadata, LocalResource.LSMBTreeResource);
TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
@@ -104,10 +105,9 @@
spec,
numSecondaryKeys,
new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, storageProperties
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
// Connect the operators.
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
index 6717b4d..9f80568 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
@@ -23,7 +23,12 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
+import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.common.context.TransactionSubsystemProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -39,8 +44,11 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.AndDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.IsNullDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.NotDescriptor;
+import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -63,6 +71,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IJobletEventListenerFactory;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
@@ -71,6 +80,7 @@
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -269,18 +279,26 @@
int[] lowKeyFields = null;
// +Infinity
int[] highKeyFields = null;
+ ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+ JobId jobId = JobIdFactory.generateJobId();
+ metadataProvider.setJobId(jobId);
+ boolean isWriteTransaction = metadataProvider.isWriteTransaction();
+ IJobletEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(jobId, isWriteTransaction);
+ spec.setJobletEventListenerFactory(jobEventListenerFactory);
+
+ ISearchOperationCallbackFactory searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(
+ jobId, dataset.getDatasetId(), primaryBloomFilterKeyFields, txnSubsystemProvider,
+ ResourceType.LSM_BTREE);
AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
- .getBloomFilterFalsePositiveRate()), false,
- NoOpOperationCallbackFactory.INSTANCE);
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new PrimaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate()), false, searchCallbackFactory);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
primaryPartitionConstraint);
return primarySearchOp;
@@ -333,11 +351,10 @@
fieldPermutation[i] = i;
}
TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, secondaryFileSplitProvider,
- secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
- fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
- NoOpOperationCallbackFactory.INSTANCE);
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
+ secondaryBloomFilterKeyFields, fieldPermutation, fillFactor, false, numElementsHint, false,
+ dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
secondaryPartitionConstraint);
return treeIndexBulkLoadOp;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
index 17590c5..72d0d70 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
@@ -22,11 +22,13 @@
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.asterix.runtime.formats.FormatUtils;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.resource.LSMInvertedIndexLocalResourceMetadata;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -176,8 +178,8 @@
IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
LSMInvertedIndexCreateOperatorDescriptor invIndexCreateOp = new LSMInvertedIndexCreateOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, secondaryFileSplitProvider,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory,
localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexCreateOp,
@@ -267,8 +269,8 @@
IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
spec, fieldPermutation, false, numElementsHint, false,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, secondaryFileSplitProvider,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory,
NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
@@ -280,17 +282,17 @@
AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
if (!isPartitioned) {
return new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate());
} else {
return new PartitionedLSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate());
}
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
index 17632aa..4d8118d 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
@@ -21,6 +21,7 @@
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -30,6 +31,7 @@
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadata;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -84,15 +86,14 @@
localResourceMetadata, LocalResource.LSMRTreeResource);
TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, null,
new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
- keyType, secondaryComparatorFactories.length), storageProperties
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
+ secondaryComparatorFactories.length), storageProperties
.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
NoOpOperationCallbackFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
@@ -169,17 +170,14 @@
AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
// Create secondary RTree bulk load op.
- TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
- spec,
- numNestedSecondaryKeyFields,
- new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
- primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
- keyType, secondaryComparatorFactories.length), storageProperties
- .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+ TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec,
+ numNestedSecondaryKeyFields, new LSMRTreeDataflowHelperFactory(valueProviderFactories,
+ RTreePolicyType.RTREE, primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
+ AqlMetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
+ storageProperties.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
// Connect the operators.
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterix-app/src/main/resources/asterix-build-configuration.xml b/asterix-app/src/main/resources/asterix-build-configuration.xml
index 6a6332d..9c4d15b 100644
--- a/asterix-app/src/main/resources/asterix-build-configuration.xml
+++ b/asterix-app/src/main/resources/asterix-build-configuration.xml
@@ -1,50 +1,40 @@
-<!--
- ! Copyright 2009-2013 by The Regents of the University of California
- ! Licensed under the Apache License, Version 2.0 (the "License");
- ! you may not use this file except in compliance with the License.
- ! you may obtain a copy of the License from
- !
- ! http://www.apache.org/licenses/LICENSE-2.0
- !
- ! Unless required by applicable law or agreed to in writing, software
- ! distributed under the License is distributed on an "AS IS" BASIS,
- ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ! See the License for the specific language governing permissions and
- ! limitations under the License.
- !-->
+<!-- ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License"); ! you may
+ not use this file except in compliance with the License. ! you may obtain
+ a copy of the License from ! ! http://www.apache.org/licenses/LICENSE-2.0
+ ! ! Unless required by applicable law or agreed to in writing, software !
+ distributed under the License is distributed on an "AS IS" BASIS, ! WITHOUT
+ WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ! See the
+ License for the specific language governing permissions and ! limitations
+ under the License. ! -->
<asterixConfiguration xmlns="asterixconf">
- <metadataNode>nc1</metadataNode>
- <store>
- <ncId>nc1</ncId>
- <storeDirs>nc1data</storeDirs>
- </store>
- <store>
- <ncId>nc2</ncId>
- <storeDirs>nc2data</storeDirs>
- </store>
- <transactionLogDir>
- <ncId>nc1</ncId>
- <txnLogDirPath>target/txnLogDir/nc1</txnLogDirPath>
- </transactionLogDir>
- <transactionLogDir>
- <ncId>nc2</ncId>
- <txnLogDirPath>target/txnLogDir/nc2</txnLogDirPath>
- </transactionLogDir>
- <property>
- <name>log.level</name>
- <value>WARNING</value>
- <description>Log level for running tests/build</description>
- </property>
- <property>
+ <metadataNode>nc1</metadataNode>
+ <store>
+ <ncId>nc1</ncId>
+ <storeDirs>nc1data</storeDirs>
+ </store>
+ <store>
+ <ncId>nc2</ncId>
+ <storeDirs>nc2data</storeDirs>
+ </store>
+ <transactionLogDir>
+ <ncId>nc1</ncId>
+ <txnLogDirPath>target/txnLogDir/nc1</txnLogDirPath>
+ </transactionLogDir>
+ <transactionLogDir>
+ <ncId>nc2</ncId>
+ <txnLogDirPath>target/txnLogDir/nc2</txnLogDirPath>
+ </transactionLogDir>
+ <property>
+ <name>log.level</name>
+ <value>WARNING</value>
+ <description>Log level for running tests/build</description>
+ </property>
+ <property>
<name>storage.memorycomponent.numpages</name>
<value>8</value>
<description>The number of pages to allocate for a memory component.
(Default = 8)
</description>
</property>
- <property>
- <name>txn.log.groupcommitinterval</name>
- <value>1</value>
- <description>The group commit wait time in milliseconds.</description>
- </property>
</asterixConfiguration>
diff --git a/asterix-app/src/test/resources/optimizerts/queries/noncollocated.aql b/asterix-app/src/test/resources/optimizerts/queries/collocated.aql
similarity index 78%
rename from asterix-app/src/test/resources/optimizerts/queries/noncollocated.aql
rename to asterix-app/src/test/resources/optimizerts/queries/collocated.aql
index 26b5ab0..bcd0a0d 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/noncollocated.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/collocated.aql
@@ -16,15 +16,11 @@
lottery_numbers: {{int32}}
}
-create nodegroup group1 if not exists on nc1, nc2;
-
-create nodegroup group2 if not exists on nc2;
-
create dataset Users(UserType)
- primary key uid on group1;
+ primary key uid;
create dataset Visitors(VisitorType)
- primary key vid on group2;
+ primary key vid;
write output to nc1:"/tmp/fuzzy1.adm";
diff --git a/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset-with-index.aql b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset-with-index.aql
new file mode 100644
index 0000000..4643eb6
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset-with-index.aql
@@ -0,0 +1,33 @@
+/*
+ * Test case Name : insert-and-scan-dataset-with-index.aql
+ * Description : This test is intended to test inserting into a dataset that has a secondary index and scan
+ * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+create type test.Emp as closed {
+id:int32,
+fname:string,
+lname:string,
+age:int32,
+dept:string
+}
+
+create dataset test.employee(Emp) primary key id;
+
+create index idx_employee_first_name on test.employee(fname);
+
+insert into dataset test.employee (
+for $x in dataset test.employee
+return {
+ "id": $x.id + 10000,
+ "fname": $x.fname,
+ "lname": $x.lname,
+ "age": $x.age,
+ "dept": $x.dept
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql
new file mode 100644
index 0000000..52c4384
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql
@@ -0,0 +1,26 @@
+/*
+ * Test case Name : insert-and-scan-dataset.aql
+ * Description : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type myDataType as open {
+ id: int32
+}
+
+create dataset myData(myDataType)
+ primary key id;
+
+insert into dataset myData (
+for $x in dataset myData
+return {
+ "id": $x.id + 1
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-joined-datasets.aql b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-joined-datasets.aql
new file mode 100644
index 0000000..a8a27ca
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-joined-datasets.aql
@@ -0,0 +1,32 @@
+/*
+ * Test case Name : insert-and-scan-joined-datasets.aql
+ * Description : This test is intended to test inserting into a dataset where the incoming stream
+ is involve a join operation that has the same dataset. We insert a materializing to prevent the
+ possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type myDataType as open {
+ id: int32
+}
+
+create dataset myData(myDataType)
+ primary key id;
+
+create dataset myData2(myDataType)
+ primary key id;
+
+insert into dataset myData (
+for $x in dataset myData2
+for $y in dataset myData
+where $x.id = $y.id
+return {
+ "id": $x.id + 1
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/q1_pricing_summary_report_nt.aql b/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt.aql
similarity index 100%
rename from asterix-app/src/test/resources/optimizerts/queries/q1_pricing_summary_report_nt.aql
rename to asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt.aql
diff --git a/asterix-app/src/test/resources/optimizerts/queries/q3_shipping_priority.aql b/asterix-app/src/test/resources/optimizerts/queries/q03_shipping_priority.aql
similarity index 100%
rename from asterix-app/src/test/resources/optimizerts/queries/q3_shipping_priority.aql
rename to asterix-app/src/test/resources/optimizerts/queries/q03_shipping_priority.aql
diff --git a/asterix-app/src/test/resources/optimizerts/queries/q5_local_supplier_volume.aql b/asterix-app/src/test/resources/optimizerts/queries/q05_local_supplier_volume.aql
similarity index 100%
rename from asterix-app/src/test/resources/optimizerts/queries/q5_local_supplier_volume.aql
rename to asterix-app/src/test/resources/optimizerts/queries/q05_local_supplier_volume.aql
diff --git a/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql b/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql
new file mode 100644
index 0000000..96b20a4
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql
@@ -0,0 +1,149 @@
+drop dataverse q08_group_by if exists;
+
+create dataverse q08_group_by;
+
+use dataverse q08_group_by;
+
+create type LineItemType as closed {
+ l_orderkey: int32,
+ l_partkey: int32,
+ l_suppkey: int32,
+ l_linenumber: int32,
+ l_quantity: double,
+ l_extendedprice: double,
+ l_discount: double,
+ l_tax: double,
+ l_returnflag: string,
+ l_linestatus: string,
+ l_shipdate: string,
+ l_commitdate: string,
+ l_receiptdate: string,
+ l_shipinstruct: string,
+ l_shipmode: string,
+ l_comment: string
+}
+
+create type OrderType as closed {
+ o_orderkey: int32,
+ o_custkey: int32,
+ o_orderstatus: string,
+ o_totalprice: double,
+ o_orderdate: string,
+ o_orderpriority: string,
+ o_clerk: string,
+ o_shippriority: int32,
+ o_comment: string
+}
+
+create type CustomerType as closed {
+ c_custkey: int32,
+ c_name: string,
+ c_address: string,
+ c_nationkey: int32,
+ c_phone: string,
+ c_acctbal: double,
+ c_mktsegment: string,
+ c_comment: string
+}
+
+create type SupplierType as closed {
+ s_suppkey: int32,
+ s_name: string,
+ s_address: string,
+ s_nationkey: int32,
+ s_phone: string,
+ s_acctbal: double,
+ s_comment: string
+}
+
+create type NationType as closed {
+ n_nationkey: int32,
+ n_name: string,
+ n_regionkey: int32,
+ n_comment: string
+}
+
+create type RegionType as closed {
+ r_regionkey: int32,
+ r_name: string,
+ r_comment: string
+}
+
+create type PartType as closed {
+ p_partkey: int32,
+ p_name: string,
+ p_mfgr: string,
+ p_brand: string,
+ p_type: string,
+ p_size: int32,
+ p_container: string,
+ p_retailprice: double,
+ p_comment: string
+}
+
+create dataset LineItem(LineItemType)
+ primary key l_orderkey, l_linenumber;
+create dataset Orders(OrderType)
+ primary key o_orderkey;
+create dataset Customer(CustomerType)
+ primary key c_custkey;
+create dataset Supplier(SupplierType)
+ primary key s_suppkey;
+create dataset Nation(NationType)
+ primary key n_nationkey;
+create dataset Region(RegionType)
+ primary key r_regionkey;
+create dataset Part(PartType)
+ primary key p_partkey;
+
+for $s in dataset("Supplier")
+ for $lnrcop in (
+ for $lnrco in (
+ for $l in dataset('LineItem')
+ for $nrco in (
+ for $o in dataset('Orders')
+ for $nrc in (
+ for $c in dataset('Customer')
+ for $nr in (
+ for $n1 in dataset('Nation')
+ for $r1 in dataset('Region')
+ where $n1.n_regionkey = $r1.r_regionkey and $r1.r_name = 'AMERICA'
+ return { "n_nationkey": $n1.n_nationkey }
+ )
+ where $c.c_nationkey = $nr.n_nationkey
+ return { "c_custkey": $c.c_custkey }
+ )
+ where $nrc.c_custkey = $o.o_custkey
+ return {
+ "o_orderdate" : $o.o_orderdate,
+ "o_orderkey": $o.o_orderkey
+ }
+ )
+ where $l.l_orderkey = $nrco.o_orderkey
+ and $nrco.o_orderdate >= '1995-01-01'
+ and $nrco.o_orderdate <= '1996-12-31'
+ return {
+ "o_orderdate": $nrco.o_orderdate,
+ "l_partkey": $l.l_partkey,
+ "l_discount": $l.l_discount,
+ "l_extendedprice": $l.l_extendedprice,
+ "l_suppkey": $l.l_suppkey
+ }
+ )
+ for $p in dataset("Part")
+ where $p.p_partkey = $lnrco.l_partkey and $p.p_type = 'ECONOMY ANODIZED STEEL'
+ return {
+ "o_orderdate": $lnrco.o_orderdate,
+ "l_discount": $lnrco.l_discount,
+ "l_extendedprice": $lnrco.l_extendedprice,
+ "l_suppkey": $lnrco.l_suppkey
+ }
+ )
+ where $s.s_suppkey = $lnrcop.l_suppkey
+ return {
+ "o_orderdate": $lnrcop.o_orderdate,
+ "l_discount": $lnrcop.l_discount,
+ "l_extendedprice": $lnrcop.l_extendedprice,
+ "l_suppkey": $lnrcop.l_suppkey,
+ "s_nationkey": $s.s_nationkey
+ }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql b/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql
new file mode 100644
index 0000000..3c9dc47
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql
@@ -0,0 +1,153 @@
+drop dataverse q09_group_by if exists;
+
+create dataverse q09_group_by;
+
+use dataverse q09_group_by;
+
+create type LineItemType as closed {
+ l_orderkey: int32,
+ l_partkey: int32,
+ l_suppkey: int32,
+ l_linenumber: int32,
+ l_quantity: int32,
+ l_extendedprice: double,
+ l_discount: double,
+ l_tax: double,
+ l_returnflag: string,
+ l_linestatus: string,
+ l_shipdate: string,
+ l_commitdate: string,
+ l_receiptdate: string,
+ l_shipinstruct: string,
+ l_shipmode: string,
+ l_comment: string
+}
+
+create type OrderType as closed {
+ o_orderkey: int32,
+ o_custkey: int32,
+ o_orderstatus: string,
+ o_totalprice: double,
+ o_orderdate: string,
+ o_orderpriority: string,
+ o_clerk: string,
+ o_shippriority: int32,
+ o_comment: string
+}
+
+create type CustomerType as closed {
+ c_custkey: int32,
+ c_name: string,
+ c_address: string,
+ c_nationkey: int32,
+ c_phone: string,
+ c_acctbal: double,
+ c_mktsegment: string,
+ c_comment: string
+}
+
+create type SupplierType as closed {
+ s_suppkey: int32,
+ s_name: string,
+ s_address: string,
+ s_nationkey: int32,
+ s_phone: string,
+ s_acctbal: double,
+ s_comment: string
+}
+
+create type NationType as closed {
+ n_nationkey: int32,
+ n_name: string,
+ n_regionkey: int32,
+ n_comment: string
+}
+
+create type RegionType as closed {
+ r_regionkey: int32,
+ r_name: string,
+ r_comment: string
+}
+
+create type PartType as closed {
+ p_partkey: int32,
+ p_name: string,
+ p_mfgr: string,
+ p_brand: string,
+ p_type: string,
+ p_size: int32,
+ p_container: string,
+ p_retailprice: double,
+ p_comment: string
+}
+
+create type PartSuppType as closed {
+ ps_partkey: int32,
+ ps_suppkey: int32,
+ ps_availqty: int32,
+ ps_supplycost: double,
+ ps_comment: string
+}
+
+create dataset LineItem(LineItemType)
+ primary key l_orderkey, l_linenumber;
+create dataset Orders(OrderType)
+ primary key o_orderkey;
+create dataset Supplier(SupplierType)
+ primary key s_suppkey;
+create dataset Region(RegionType)
+ primary key r_regionkey;
+create dataset Nation(NationType)
+ primary key n_nationkey;
+create dataset Part(PartType)
+ primary key p_partkey;
+create dataset Partsupp(PartSuppType)
+ primary key ps_partkey, ps_suppkey;
+create dataset Customer(CustomerType)
+ primary key c_custkey;
+
+for $p in dataset('Part')
+ for $l2 in (
+ for $ps in dataset('Partsupp')
+ for $l1 in (
+ for $s1 in (
+ for $s in dataset('Supplier')
+ for $n in dataset('Nation')
+ where $n.n_nationkey = $s.s_nationkey
+ return {
+ "s_suppkey": $s.s_suppkey,
+ "n_name": $n.n_name
+ }
+ )
+ for $l in dataset('LineItem')
+ where $s1.s_suppkey = $l.l_suppkey
+ return {
+ "l_suppkey": $l.l_suppkey,
+ "l_extendedprice": $l.l_extendedprice,
+ "l_discount": $l.l_discount,
+ "l_quantity": $l.l_quantity,
+ "l_partkey": $l.l_partkey,
+ "l_orderkey": $l.l_orderkey,
+ "n_name": $s1.n_name
+ }
+ )
+ where $ps.ps_suppkey = $l1.l_suppkey and $ps.ps_partkey = $l1.l_partkey
+ return {
+ "l_extendedprice": $l1.l_extendedprice,
+ "l_discount": $l1.l_discount,
+ "l_quantity": $l1.l_quantity,
+ "l_partkey": $l1.l_partkey,
+ "l_orderkey": $l1.l_orderkey,
+ "n_name": $l1.n_name,
+ "ps_supplycost": $ps.ps_supplycost
+ }
+ )
+ where contains($p.p_name, 'green') and $p.p_partkey = $l2.l_partkey
+ return {
+ "l_extendedprice": $l2.l_extendedprice,
+ "l_discount": $l2.l_discount,
+ "l_quantity": $l2.l_quantity,
+ "l_orderkey": $l2.l_orderkey,
+ "n_name": $l2.n_name,
+ "ps_supplycost": $l2.ps_supplycost
+ }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/noncollocated.plan b/asterix-app/src/test/resources/optimizerts/results/collocated.plan
similarity index 88%
rename from asterix-app/src/test/resources/optimizerts/results/noncollocated.plan
rename to asterix-app/src/test/resources/optimizerts/results/collocated.plan
index e92a84c..116ff8a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/noncollocated.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/collocated.plan
@@ -5,14 +5,14 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- HYBRID_HASH_JOIN [$$10][$$11] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$10] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$11] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan b/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan
index 0c7b95d..8ae274a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan
@@ -73,8 +73,8 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$60] |PARTITIONED|
- -- STREAM_PROJECT |UNPARTITIONED|
- -- ASSIGN |UNPARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- SPLIT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset-with-index.plan b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset-with-index.plan
new file mode 100644
index 0000000..8bc296b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset-with-index.plan
@@ -0,0 +1,20 @@
+-- COMMIT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INDEX_INSERT_DELETE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INSERT_DELETE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- MATERIALIZE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$10] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
new file mode 100644
index 0000000..e11e2a8
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
@@ -0,0 +1,15 @@
+-- COMMIT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INSERT_DELETE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- MATERIALIZE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$6] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-joined-datasets.plan b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-joined-datasets.plan
new file mode 100644
index 0000000..88f256e
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-joined-datasets.plan
@@ -0,0 +1,25 @@
+-- COMMIT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INSERT_DELETE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- MATERIALIZE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$11] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$14][$$15] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/q1_pricing_summary_report_nt.plan b/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt.plan
similarity index 100%
rename from asterix-app/src/test/resources/optimizerts/results/q1_pricing_summary_report_nt.plan
rename to asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt.plan
diff --git a/asterix-app/src/test/resources/optimizerts/results/q3_shipping_priority.plan b/asterix-app/src/test/resources/optimizerts/results/q03_shipping_priority.plan
similarity index 100%
rename from asterix-app/src/test/resources/optimizerts/results/q3_shipping_priority.plan
rename to asterix-app/src/test/resources/optimizerts/results/q03_shipping_priority.plan
diff --git a/asterix-app/src/test/resources/optimizerts/results/q5_local_supplier_volume.plan b/asterix-app/src/test/resources/optimizerts/results/q05_local_supplier_volume.plan
similarity index 100%
rename from asterix-app/src/test/resources/optimizerts/results/q5_local_supplier_volume.plan
rename to asterix-app/src/test/resources/optimizerts/results/q05_local_supplier_volume.plan
diff --git a/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan b/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
new file mode 100644
index 0000000..fefdfcb
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
@@ -0,0 +1,78 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$104] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$104] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$101][$$85] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$101] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$79][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$79] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$96][$$82] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$96] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$82] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$92][$$83] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$92] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$89][$$84] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$89] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan b/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
new file mode 100644
index 0000000..8a08e39
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
@@ -0,0 +1,55 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$62][$$80] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$80] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$64, $$63][$$69, $$80] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$80, $$69] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$65][$$69] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$65] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$73][$$66] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$73] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$69] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
index cba987c..c241913 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
@@ -3,7 +3,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- MATERIALIZE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$20] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -11,4 +11,4 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
index e1852af..51800a2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
@@ -10,7 +10,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$15(ASC)] |PARTITIONED|
+ -- MATERIALIZE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,4 +20,4 @@
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
index a5265c8..3a4bc84 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
@@ -3,7 +3,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
+ -- MATERIALIZE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$22] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -13,4 +13,4 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
index 8057666..cc9a865 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
@@ -12,15 +12,13 @@
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$7(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$7] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$7] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
index 51d3060..b34944d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
@@ -2,15 +2,13 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$7(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$7] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$7] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue531_string_min_max/issue531_string_min_max.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue531_string_min_max/issue531_string_min_max.1.ddl.aql
new file mode 100644
index 0000000..04c1e6f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue531_string_min_max/issue531_string_min_max.1.ddl.aql
@@ -0,0 +1,18 @@
+/**
+ * issue531_string_min_max
+ *
+ * Purpose: test the support of string values for min and max aggregation function
+ * Result: success
+ *
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TestType as open{
+id:int32,
+name:string
+}
+
+create dataset t1(TestType) primary key id;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue531_string_min_max/issue531_string_min_max.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue531_string_min_max/issue531_string_min_max.2.update.aql
new file mode 100644
index 0000000..2e0bb6c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue531_string_min_max/issue531_string_min_max.2.update.aql
@@ -0,0 +1,18 @@
+/**
+ * issue531_string_min_max
+ *
+ * Purpose: test the support of string values for min and max aggregation function
+ * Result: success
+ *
+ */
+
+use dataverse test;
+
+insert into dataset t1({"id":5,"name":"Smith"});
+insert into dataset t1({"id":12,"name":"Roger"});
+insert into dataset t1({"id":67,"name":"Kevin"});
+insert into dataset t1({"id":32,"name":"Bob"});
+insert into dataset t1({"id":89,"name":"John"});
+insert into dataset t1({"id":10,"name":"Alex"});
+insert into dataset t1({"id":37,"name":"Calvin"});
+insert into dataset t1({"id":98,"name":"Susan"});
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.aql
new file mode 100644
index 0000000..1fa706c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.aql
@@ -0,0 +1,13 @@
+/**
+ * issue531_string_min_max
+ *
+ * Purpose: test the support of string values for min and max aggregation function
+ * Result: success
+ *
+ */
+
+use dataverse test;
+
+{"min": min(for $l in dataset t1
+return $l.name), "max": max(for $l in dataset t1
+return $l.name)}
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/boolean/and_null/and_null.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/boolean/and_null/and_null.3.query.aql
index 830ffe9..f49dbd7 100644
--- a/asterix-app/src/test/resources/runtimets/queries/boolean/and_null/and_null.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/boolean/and_null/and_null.3.query.aql
@@ -1,5 +1,5 @@
use dataverse test;
-let $x := boolean("true")
+let $x := true
let $y := null
return $x and $y
diff --git a/asterix-app/src/test/resources/runtimets/queries/boolean/and_null_false/and_null_false.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/boolean/and_null_false/and_null_false.3.query.aql
index 5e646a8..371246d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/boolean/and_null_false/and_null_false.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/boolean/and_null_false/and_null_false.3.query.aql
@@ -1,5 +1,5 @@
use dataverse test;
-let $x := boolean("false")
+let $x := false
let $y := null
return $x and $y
diff --git a/asterix-app/src/test/resources/runtimets/queries/custord/denorm-cust-order_02/denorm-cust-order_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/custord/denorm-cust-order_02/denorm-cust-order_02.2.update.aql
index 8faaa98..a3f070b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/custord/denorm-cust-order_02/denorm-cust-order_02.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/custord/denorm-cust-order_02/denorm-cust-order_02.2.update.aql
@@ -2,11 +2,11 @@
load dataset Customers2
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/custord-tiny/customer-tiny.adm"),("format"="adm")) pre-sorted;
+(("path"="nc1://data/custord-tiny/customer-tiny.adm"),("format"="adm"));
load dataset Orders2
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/custord-tiny/order-tiny.adm"),("format"="adm")) pre-sorted;
+(("path"="nc1://data/custord-tiny/order-tiny.adm"),("format"="adm"));
insert into dataset CustomerOrders2 (
diff --git a/asterix-app/src/test/resources/runtimets/queries/custord/load-test/load-test.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/custord/load-test/load-test.2.update.aql
index a395f03..c0ab30f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/custord/load-test/load-test.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/custord/load-test/load-test.2.update.aql
@@ -2,5 +2,5 @@
load dataset c1
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/custord-tiny/customer-tiny.adm"),("format"="adm")) pre-sorted;
+(("path"="nc1://data/custord-tiny/customer-tiny.adm"),("format"="adm"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.3.query.aql
index 6b09eec..38f292b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.3.query.aql
@@ -1,11 +1,10 @@
/*
* Description : This test case is to verify the fix for issue443
- : https://code.google.com/p/asterixdb/issues/detail?id=443
- * Expected Res : Fail
+ * https://code.google.com/p/asterixdb/issues/detail?id=443
+ * Expected Res : Success
* Date : 22th May 2013
*/
-
for $a in [ {"f" : 19, "g": 1} , {"f" : 12, "g": 2} , {"f" : 10, "g": 1} , {"f" : 17, "g": 1}, {"f" : 12, "g": 4} ]
distinct by $a.f
-return $a
\ No newline at end of file
+return $a
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.aql
new file mode 100644
index 0000000..6ded7a7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.aql
@@ -0,0 +1,22 @@
+/*
+ * Test case Name : insert-and-scan-dataset-with-index.aql
+ * Description : This test is intended to test inserting into a dataset that has a secondary index and scan
+ * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+create type test.Emp as closed {
+id:int32,
+fname:string,
+lname:string,
+age:int32,
+dept:string
+}
+
+create dataset test.employee(Emp) primary key id;
+
+create index idx_employee_first_name on test.employee(fname);
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
new file mode 100644
index 0000000..29f6242
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * Test case Name : insert-and-scan-dataset-with-index.aql
+ * Description : This test is intended to test inserting into a dataset that has a secondary index and scan
+ * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+use dataverse test;
+
+load dataset test.employee
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/names.adm"),("format"="delimited-text"),("delimiter"="|"));
+
+
+insert into dataset test.employee (
+for $x in dataset test.employee
+return {
+ "id": $x.id + 10000,
+ "fname": $x.fname,
+ "lname": $x.lname,
+ "age": $x.age,
+ "dept": $x.dept
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.aql
new file mode 100644
index 0000000..ae259a9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Test case Name : insert-and-scan-dataset-with-index.aql
+ * Description : This test is intended to test inserting into a dataset that has a secondary index and scan
+ * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+use dataverse test;
+
+for $l in dataset('test.employee')
+order by $l.id
+return $l
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.aql
new file mode 100644
index 0000000..fc04212
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Test case Name : insert-and-scan-dataset.aql
+ * Description : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type myDataType as open {
+ id: int32
+}
+
+create dataset myData(myDataType)
+ primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.2.update.aql
new file mode 100644
index 0000000..58e0c38
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.2.update.aql
@@ -0,0 +1,21 @@
+/*
+ * Test case Name : insert-and-scan-dataset.aql
+ * Description : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+use dataverse test;
+
+load dataset myData
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/odd-numbers.adm"),("format"="adm"))pre-sorted;
+
+
+insert into dataset myData (
+for $x in dataset myData
+return {
+ "id": $x.id + 1
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.3.query.aql
new file mode 100644
index 0000000..f9af922
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Test case Name : insert-and-scan-dataset.aql
+ * Description : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+use dataverse test;
+
+for $c in dataset('myData')
+order by $c.id
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.ddl.aql
new file mode 100644
index 0000000..adf6239
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.ddl.aql
@@ -0,0 +1,23 @@
+/*
+ * Test case Name : insert-and-scan-joined-datasets.aql
+ * Description : This test is intended to test inserting into a dataset where the incoming stream
+ is involve a join operation that has the same dataset. We insert a materializing to prevent the
+ possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type myDataType as open {
+ id: int32
+}
+
+create dataset myData(myDataType)
+ primary key id;
+
+create dataset myData2(myDataType)
+ primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.2.update.aql
new file mode 100644
index 0000000..6b9e1b6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.2.update.aql
@@ -0,0 +1,27 @@
+/*
+ * Test case Name : insert-and-scan-joined-datasets.aql
+ * Description : This test is intended to test inserting into a dataset where the incoming stream
+ is involve a join operation that has the same dataset. We insert a materializing to prevent the
+ possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+use dataverse test;
+
+load dataset myData
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/odd-numbers.adm"),("format"="adm"))pre-sorted;
+
+load dataset myData2
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/odd-numbers-2.adm"),("format"="adm"))pre-sorted;
+
+insert into dataset myData (
+for $x in dataset myData2
+for $y in dataset myData
+where $x.id = $y.id
+return {
+ "id": $x.id + 1
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.3.query.aql
new file mode 100644
index 0000000..3614e1b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.3.query.aql
@@ -0,0 +1,14 @@
+/*
+ * Test case Name : insert-and-scan-joined-datasets.aql
+ * Description : This test is intended to test inserting into a dataset where the incoming stream
+ is involve a join operation that has the same dataset. We insert a materializing to prevent the
+ possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+use dataverse test;
+
+for $c in dataset('myData')
+order by $c.id
+return $c
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/opentype-insert2/opentype-insert2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/opentype-insert2/opentype-insert2.2.update.aql
index ec357bf..be95b39 100644
--- a/asterix-app/src/test/resources/runtimets/queries/dml/opentype-insert2/opentype-insert2.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/opentype-insert2/opentype-insert2.2.update.aql
@@ -7,5 +7,5 @@
use dataverse test;
-insert into dataset testds( for $i in range(1,10) return { "id":$i,"name":"John Doe" });
+insert into dataset testds( for $i in range(1, 10) return { "id":$i,"name":"John Doe" });
diff --git a/asterix-app/src/test/resources/runtimets/queries/failure/q1_pricing_summary_report_failure/q1_pricing_summary_report_failure.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/failure/q01_pricing_summary_report_failure/q01_pricing_summary_report_failure.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/failure/q1_pricing_summary_report_failure/q1_pricing_summary_report_failure.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/failure/q01_pricing_summary_report_failure/q01_pricing_summary_report_failure.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/failure/q1_pricing_summary_report_failure/q1_pricing_summary_report_failure.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/failure/q01_pricing_summary_report_failure/q01_pricing_summary_report_failure.2.update.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/failure/q1_pricing_summary_report_failure/q1_pricing_summary_report_failure.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/failure/q01_pricing_summary_report_failure/q01_pricing_summary_report_failure.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/failure/q1_pricing_summary_report_failure/q1_pricing_summary_report_failure.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/failure/q01_pricing_summary_report_failure/q01_pricing_summary_report_failure.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/failure/q1_pricing_summary_report_failure/q1_pricing_summary_report_failure.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/failure/q01_pricing_summary_report_failure/q01_pricing_summary_report_failure.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.1.ddl.aql
new file mode 100644
index 0000000..e890942
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.1.ddl.aql
@@ -0,0 +1,22 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+drop dataverse temp if exists;
+create dataverse temp
+use dataverse temp;
+
+create type test as closed {
+ id: int32,
+ float: float?,
+ double: double?,
+ date: string?,
+ time: string?,
+ datetime: string?
+};
+
+create dataset testds (test)
+primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.2.update.aql
new file mode 100644
index 0000000..c3161d5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.2.update.aql
@@ -0,0 +1,12 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+use dataverse temp;
+
+load dataset testds
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/csv/sample_01.csv"),("format"="delimited-text"),("delimiter"=","));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.3.query.aql
new file mode 100644
index 0000000..efa6dbc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_01/csv_01.3.query.aql
@@ -0,0 +1,18 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+use dataverse temp;
+
+for $i in dataset testds
+order by $i.id
+return { "id": $i.id,
+ "float": $i.float,
+ "double": $i.double,
+ "date-before": $i.date, "date-after": date($i.date),
+ "time-before": $i.time, "time-after": time($i.time),
+ "datetime-before": $i.datetime, "datetime-after": datetime($i.datetime)
+ }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.1.ddl.aql
new file mode 100644
index 0000000..b6884a8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.1.ddl.aql
@@ -0,0 +1,22 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+drop dataverse temp if exists;
+create dataverse temp
+use dataverse temp;
+
+create type test as closed {
+ id: int32,
+ float: float?,
+ double: double?,
+ date: string,
+ time: string,
+ datetime: string
+};
+
+create dataset testds (test)
+primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.2.update.aql
new file mode 100644
index 0000000..c3161d5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.2.update.aql
@@ -0,0 +1,12 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+use dataverse temp;
+
+load dataset testds
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/csv/sample_01.csv"),("format"="delimited-text"),("delimiter"=","));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.3.query.aql
new file mode 100644
index 0000000..1299235
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/csv_02/csv_02.3.query.aql
@@ -0,0 +1,18 @@
+/**
+ *
+ * CSV file loading test
+ * Expected result: success
+ *
+ */
+
+use dataverse temp;
+
+for $i in dataset testds
+order by $i.id
+return { "id": $i.id,
+ "float": $i.float,
+ "double": $i.double,
+ "date-string": $i.date,
+ "time-string": $i.time,
+ "datetime-string": $i.datetime
+ }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql
index f0671ca..8a52ab9 100644
--- a/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql
@@ -8,4 +8,4 @@
load dataset Customers
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/custord-tiny/customer-tiny-neg.adm"),("format"="adm")) pre-sorted;
\ No newline at end of file
+(("path"="nc1://data/custord-tiny/customer-tiny-neg.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/nestrecords/nestrecord/nestrecord.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/nestrecords/nestrecord/nestrecord.2.update.aql
index bca3929..ac1a9f1 100644
--- a/asterix-app/src/test/resources/runtimets/queries/nestrecords/nestrecord/nestrecord.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/nestrecords/nestrecord/nestrecord.2.update.aql
@@ -7,7 +7,7 @@
use dataverse testdv2;
insert into dataset testds (
-{ "id": "001", "name": "Person One", "address": {"street": "3019 DBH", "city": "Irvine", "zip": 92697} }
+{ "id": "001", "name": "Person One", "address": {"street": "3019 DBH", "city": "Irvine", "zip": 92697}}
);
insert into dataset testds (
@@ -15,10 +15,10 @@
);
insert into dataset testds (
-{ "id": "003", "name": "Person Three", "address": {"street": "2019 DBH", "city": "Irvine"} }
+{ "id": "003", "name": "Person Three", "address": {"street": "2019 DBH", "city": "Irvine"}}
);
insert into dataset testds (
-{ "id": "004", "name": "Person Four", "home": {"street": "2019 DBH", "city": {"name": "Irvine", "zip": 92697} } }
+{ "id": "004", "name": "Person Four", "home": {"street": "2019 DBH", "city": {"name": "Irvine", "zip": 92697}}}
);
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_double_01/unary-minus_double_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_double_01/unary-minus_double_01.3.query.aql
deleted file mode 100644
index 169ca39..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_double_01/unary-minus_double_01.3.query.aql
+++ /dev/null
@@ -1,7 +0,0 @@
-use dataverse test;
-
-let $c1 := double("-20.56e-30")
-let $c2 := double("NaN")
-let $c3 := double("INF")
-let $c4 := double("-INF")
-return {"double1": numeric-unary-minus($c1),"double2": numeric-unary-minus($c2),"double3": numeric-unary-minus($c3),"double4": numeric-unary-minus($c4)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_float_01/unary-minus_float_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_float_01/unary-minus_float_01.1.ddl.aql
deleted file mode 100644
index 754ea81..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_float_01/unary-minus_float_01.1.ddl.aql
+++ /dev/null
@@ -1,3 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_float_01/unary-minus_float_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_float_01/unary-minus_float_01.2.update.aql
deleted file mode 100644
index e69de29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_float_01/unary-minus_float_01.2.update.aql
+++ /dev/null
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_float_01/unary-minus_float_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_float_01/unary-minus_float_01.3.query.aql
deleted file mode 100644
index e5e0071..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_float_01/unary-minus_float_01.3.query.aql
+++ /dev/null
@@ -1,7 +0,0 @@
-use dataverse test;
-
-let $c1 := float("-80.20f")
-let $c2 := float("NaN")
-let $c3 := float("INF")
-let $c4 := float("-INF")
-return {"float1": numeric-unary-minus($c1),"float2": numeric-unary-minus($c2),"float3": numeric-unary-minus($c3),"float4": numeric-unary-minus($c4)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_int_01/unary-minus_int_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_int_01/unary-minus_int_01.1.ddl.aql
deleted file mode 100644
index 754ea81..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_int_01/unary-minus_int_01.1.ddl.aql
+++ /dev/null
@@ -1,3 +0,0 @@
-drop dataverse test if exists;
-create dataverse test;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_int_01/unary-minus_int_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_int_01/unary-minus_int_01.3.query.aql
deleted file mode 100644
index 660e55f..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_int_01/unary-minus_int_01.3.query.aql
+++ /dev/null
@@ -1,7 +0,0 @@
-use dataverse test;
-
-let $c1 := int8("+80")
-let $c2 := int16("160")
-let $c3 := int32("+320")
-let $c4 := int64("-640")
-return {"int8": numeric-unary-minus($c1),"int16": numeric-unary-minus($c2),"int32": numeric-unary-minus($c3),"int64": numeric-unary-minus($c4)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_double_01/unary-minus_double_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_double_01/unary-minus_double_01.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_double_01/unary-minus_double_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.2.update.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_double_01/unary-minus_double_01.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.3.query.aql
new file mode 100644
index 0000000..79de83c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.3.query.aql
@@ -0,0 +1,19 @@
+use dataverse test;
+set import-private-functions 'true';
+
+let $v1 := [ "query", "processing", "in", "multidatabase", "systems" ]
+let $v2 := [ "query", "processing", "in", "object", "oriented", "database", "systems" ]
+let $v3 := [ "dynamic", "query", "optimization", "and", "query", "processing", "in", "multidatabase", "systems", "1" ]
+let $v4 := [ "transaction", "management", "in", "multidatabase", "systems" ]
+let $v5 := [ "overview", "of", "multidatabase", "transaction", "management" ]
+
+
+let $results :=
+[
+ similarity-jaccard-check($v1, $v2, 0.5f),
+ similarity-jaccard-check($v1, $v3, 0.5f),
+ similarity-jaccard-check($v4, $v5, 0.5f)
+]
+
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/calendar_duration/calendar_duration.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/calendar_duration/calendar_duration.3.query.aql
index 016f3b3..b1c3a20 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/calendar_duration/calendar_duration.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/calendar_duration/calendar_duration.3.query.aql
@@ -8,36 +8,36 @@
let $dr3 := duration("P1Y90M")
let $dr4 := duration("-P3Y89M4089DT47382.983S")
let $cdr1 := calendar-duration-from-datetime($t1, $dr1)
-let $dt1 := add-datetime-duration($t1, $dr1)
-let $dtt1 := add-datetime-duration($t1, $cdr1)
+let $dt1 := $t1 + $dr1
+let $dtt1 := $t1 + $cdr1
let $c1 := $dt1 = $dtt1
let $cdr2 := calendar-duration-from-datetime($t1, $dr2)
-let $dt2 := add-datetime-duration($t1, $dr2)
-let $dtt2 := add-datetime-duration($t1, $cdr2)
+let $dt2 := $t1 + $dr2
+let $dtt2 := $t1 + $cdr2
let $c2 := $dt2 = $dtt2
let $cdr3 := calendar-duration-from-datetime($t1, $dr3)
-let $dt3 := add-datetime-duration($t1, $dr3)
-let $dtt3 := add-datetime-duration($t1, $cdr3)
+let $dt3 := $t1 + $dr3
+let $dtt3 := $t1 + $cdr3
let $c3 := $dt3 = $dtt3
let $cdr4 := calendar-duration-from-datetime($t1, $dr4)
-let $dt4 := add-datetime-duration($t1, $dr4)
-let $dtt4 := add-datetime-duration($t1, $cdr4)
+let $dt4 := $t1 + $dr4
+let $dtt4 := $t1 + $cdr4
let $c4 := $dt4 = $dtt4
let $cdr5 := calendar-duration-from-date($t2, $dr1)
-let $dt5 := add-date-duration($t2, $dr1)
-let $dtt5 := add-date-duration($t2, $cdr5)
+let $dt5 := $t2 + $dr1
+let $dtt5 := $t2 + $cdr5
let $c5 := $dt5 = $dtt5
let $cdr6 := calendar-duration-from-date($t2, $dr2)
-let $dt6 := add-date-duration($t2, $dr2)
-let $dtt6 := add-date-duration($t2, $cdr6)
+let $dt6 := $t2 + $dr2
+let $dtt6 := $t2 + $cdr6
let $c6 := $dt6 = $dtt6
let $cdr7 := calendar-duration-from-date($t2, $dr3)
-let $dt7 := add-date-duration($t2, $dr3)
-let $dtt7 := add-date-duration($t2, $cdr7)
+let $dt7 := $t2 + $dr3
+let $dtt7 := $t2 + $cdr7
let $c7 := $dt7 = $dtt7
let $cdr8 := calendar-duration-from-date($t2, $dr4)
-let $dt8 := add-date-duration($t2, $dr4)
-let $dtt8 := add-date-duration($t2, $cdr8)
+let $dt8 := $t2 + $dr4
+let $dtt8 := $t2 + $cdr8
let $c8 := $dt8 = $dtt8
return { "cduration1":$cdr1, "c1":$c1, "cduration2":$cdr2, "c2":$c2, "cduration3":$cdr3, "c3":$c3, "cduration4":$cdr4, "c4":$c4, "cduration5":$cdr5, "c5":$c5, "cduration6":$cdr6, "c6":$c6, "cduration7":$cdr7, "c7":$c7, "cduration8":$cdr8, "c8":$c8, "cduration-null-1": calendar-duration-from-datetime(null, $dr1), "cduration-null-2": calendar-duration-from-datetime($t1, null), "cduration-null-3": calendar-duration-from-date(null, $dr1), "cduration-null-4": calendar-duration-from-date($t2, null) }
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/date_functions/date_functions.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/date_functions/date_functions.3.query.aql
index 3927f72..e540817 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/date_functions/date_functions.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/date_functions/date_functions.3.query.aql
@@ -9,16 +9,16 @@
let $dt2 := datetime("2012-10-11T02:30:23+03:00")
let $d3 := get-date-from-datetime($dt2)
let $dr1 := duration("-P2Y1M90DT30H")
-let $d4 := add-date-duration($d1, $dr1)
-let $null3 := add-date-duration(null, $dr1)
-let $null4 := add-date-duration($d1, null)
-let $c1 := $d1 = add-date-duration($d4, subtract-date($d1, $d4))
+let $d4 := $d1 + $dr1
+let $null3 := null + $dr1
+let $null4 := $d1 + null
+let $c1 := ($d1 = ($d4 + ($d1 - $d4)))
let $dr2 := duration("P300Y900MT360000M")
-let $d5 := add-date-duration($d2, $dr2)
-let $c2 := $d2 = add-date-duration($d5, subtract-date($d2, $d5))
-let $dr3 := subtract-date($d5, $d2)
-let $dr4 := subtract-date($d4, $d1)
-let $null5 := subtract-date(null, $d2)
-let $null6 := subtract-date($d5, null)
+let $d5 := $d2 + $dr2
+let $c2 := ($d2 = ($d5 + ($d2 - $d5)))
+let $dr3 := $d5 - $d2
+let $dr4 := $d4 - $d1
+let $null5 := null - $d2
+let $null6 := $d5 - null
return { "date1": $d1, "date2": $d2, "date3": $d3, "date4": $d4, "date5": $d5, "duration1": $dr3, "duration2": $dr4, "c1": $c1, "c2": $c2, "null1": $null1, "null2": $null2, "null3": $null3, "null4": $null4, "null5": $null5, "null6": $null6 }
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/datetime_functions/datetime_functions.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/datetime_functions/datetime_functions.3.query.aql
index 3d24549..fe0c099 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/datetime_functions/datetime_functions.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/datetime_functions/datetime_functions.3.query.aql
@@ -8,12 +8,12 @@
let $dt2 := datetime-from-date-time($d1, $t1)
let $null2 := datetime-from-date-time(null, $t1)
let $null3 := datetime-from-date-time($d1, null)
-let $dr1 := subtract-datetime($dt2, $dt1)
-let $null4 := subtract-datetime(null, $dt1)
-let $null5 := subtract-datetime($dt2, null)
-let $dt3 := add-datetime-duration($dt1, $dr1)
-let $null6 := add-datetime-duration(null, $dr1)
-let $null7 := add-datetime-duration($dt1, null)
-let $c1 := $dt1 = add-datetime-duration($dt3, subtract-datetime($dt1, $dt3))
+let $dr1 := $dt2 - $dt1
+let $null4 := null - $dt1
+let $null5 := $dt2 - null
+let $dt3 := $dt1 + $dr1
+let $null6 := null + $dr1
+let $null7 := $dt1 + null
+let $c1 := $dt1 = ($dt1 - $dt3) + $dt3
return { "datetime1" : $dt1, "datetime2" : $dt2, "datetime3" : $dt3, "duration1" : $dr1, "c1" : $c1, "null1" : $null1, "null2" : $null2, "null3" : $null3, "null4" : $null4, "null5" : $null5, "null6" : $null6, "null7" : $null7 }
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/day_of_week_01/day_of_week_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/day_of_week_01/day_of_week_01.1.ddl.aql
new file mode 100644
index 0000000..f92bdd4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/day_of_week_01/day_of_week_01.1.ddl.aql
@@ -0,0 +1,7 @@
+/**
+ * day-of-week test case: test the day-of-week function
+ * Expected result: success
+ **/
+
+drop dataverse test if exists;
+create dataverse test;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/day_of_week_01/day_of_week_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/day_of_week_01/day_of_week_01.2.update.aql
new file mode 100644
index 0000000..cf8a1ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/day_of_week_01/day_of_week_01.2.update.aql
@@ -0,0 +1,4 @@
+/**
+ * day-of-week test case: test the day-of-week function
+ * Expected result: success
+ **/
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/day_of_week_01/day_of_week_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/day_of_week_01/day_of_week_01.3.query.aql
new file mode 100644
index 0000000..fc1e705
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/day_of_week_01/day_of_week_01.3.query.aql
@@ -0,0 +1,12 @@
+/**
+ * day-of-week test case: test the day-of-week function
+ * Expected result: success
+ **/
+
+use dataverse test;
+
+let $d1 := date("2013-08-06")
+let $d2 := date("-2013-08-06")
+let $dt1 := datetime("1913-08-06T15:53:28Z")
+let $dt2 := datetime("-1913-08-10T15:53:28Z")
+return { "1970-01-01": day-of-week(date("1970-01-01")), "2013-08-06": day-of-week($d1), "-2013-08-06": day-of-week($d2), "1913-08-06T15:53:28Z": day-of-week($dt1), "-1913-08-10T15:53:28Z": day-of-week($dt2), "null": day-of-week(null) }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin/interval_bin.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin/interval_bin.1.ddl.aql
new file mode 100644
index 0000000..57b3b28
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin/interval_bin.1.ddl.aql
@@ -0,0 +1,7 @@
+/**
+ * Interval_bin test case: test the interval-bin function
+ * Expected result: success
+ **/
+
+drop dataverse test if exists;
+create dataverse test;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin/interval_bin.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin/interval_bin.2.update.aql
new file mode 100644
index 0000000..e2e3afb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin/interval_bin.2.update.aql
@@ -0,0 +1,4 @@
+/**
+ * Interval_bin test case: test the interval-bin function
+ * Expected result: success
+ **/
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin/interval_bin.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin/interval_bin.3.query.aql
new file mode 100644
index 0000000..adbb2a2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin/interval_bin.3.query.aql
@@ -0,0 +1,19 @@
+/**
+ * Interval_bin test case: test the interval-bin function
+ * Expected result: success
+ **/
+
+use dataverse test;
+
+let $c1 := date("2010-10-30")
+let $c2 := datetime("-1987-11-19T23:49:23.938")
+let $c3 := time("12:23:34.930+07:00")
+
+return { "bin1": interval-bin($c1, date("1990-01-01"), year-month-duration("P1Y")),
+ "bin2": interval-bin($c1, date("-1990-01-01"), year-month-duration("P1Y")),
+ "bin3": interval-bin($c2, datetime("1990-01-01T00:00:00.000Z"), year-month-duration("P6M")),
+ "bin4": interval-bin($c2, datetime("-1990-01-01T00:00:00.000Z"), day-time-duration("PT12H")),
+ "bin5": interval-bin($c3, time("12:00:00"), day-time-duration("PT2H")),
+ "bin6": interval-bin(null, date("-0023-01-01"), year-month-duration("P6M")),
+ "bin7": interval-bin($c1, null, year-month-duration("P6M")),
+ "bin8": interval-bin($c1, date("-0023-01-01"), null) }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_0/interval_bin_gby_0.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_0/interval_bin_gby_0.1.ddl.aql
new file mode 100644
index 0000000..841d8da
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_0/interval_bin_gby_0.1.ddl.aql
@@ -0,0 +1,16 @@
+/**
+ * Interval_bin_gby test case: test the group-by using interval-bin function
+ * Expected result: success
+ **/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Schema as closed{
+id: int32,
+timestamp: datetime
+}
+
+create dataset tsdata(Schema)
+primary key id;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_0/interval_bin_gby_0.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_0/interval_bin_gby_0.2.update.aql
new file mode 100644
index 0000000..b1119e5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_0/interval_bin_gby_0.2.update.aql
@@ -0,0 +1,18 @@
+/**
+ * Interval_bin_gby test case: test the group-by using interval-bin function
+ * Expected result: success
+ **/
+use dataverse test;
+
+insert into dataset tsdata({"id": 1, "timestamp": datetime("-1987-11-19T23:49:23.938")})
+insert into dataset tsdata({"id": 2, "timestamp": datetime("-1987-11-20T00:27:13.432")})
+insert into dataset tsdata({"id": 3, "timestamp": datetime("-1987-11-18T18:00:00")})
+insert into dataset tsdata({"id": 4, "timestamp": datetime("19871119T234923938")})
+insert into dataset tsdata({"id": 5, "timestamp": datetime("1987-11-19T23:58:17.038")})
+insert into dataset tsdata({"id": 6, "timestamp": datetime("1987-11-19T23:30:00")})
+insert into dataset tsdata({"id": 7, "timestamp": datetime("1987-11-19T23:22:38")})
+insert into dataset tsdata({"id": 8, "timestamp": datetime("1988-01-21T17:28:13.900")})
+insert into dataset tsdata({"id": 9, "timestamp": datetime("-1987-11-19T23:49:23.938")})
+insert into dataset tsdata({"id": 10, "timestamp": datetime("-0987-07-01T09:35:28.039")})
+insert into dataset tsdata({"id": 11, "timestamp": datetime("2012-11-19T23:49:23.938")})
+insert into dataset tsdata({"id": 12, "timestamp": datetime("2013-11-19T23:49:23.938")})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_0/interval_bin_gby_0.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_0/interval_bin_gby_0.3.query.aql
new file mode 100644
index 0000000..776493f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_0/interval_bin_gby_0.3.query.aql
@@ -0,0 +1,11 @@
+/**
+ * Interval_bin_gby test case: test the group-by using interval-bin function
+ * Expected result: success
+ **/
+
+use dataverse test;
+
+for $i in dataset tsdata
+group by $d := interval-bin($i.timestamp, datetime("1990-01-01T00:00:00.000Z"), year-month-duration("P20Y")) with $i
+order by get-interval-start($d)
+return { "tbin": $d, "count": count($i)}
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_1/interval_bin_gby_1.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_1/interval_bin_gby_1.1.ddl.aql
new file mode 100644
index 0000000..841d8da
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_1/interval_bin_gby_1.1.ddl.aql
@@ -0,0 +1,16 @@
+/**
+ * Interval_bin_gby test case: test the group-by using interval-bin function
+ * Expected result: success
+ **/
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Schema as closed{
+id: int32,
+timestamp: datetime
+}
+
+create dataset tsdata(Schema)
+primary key id;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_1/interval_bin_gby_1.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_1/interval_bin_gby_1.2.update.aql
new file mode 100644
index 0000000..b1119e5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_1/interval_bin_gby_1.2.update.aql
@@ -0,0 +1,18 @@
+/**
+ * Interval_bin_gby test case: test the group-by using interval-bin function
+ * Expected result: success
+ **/
+use dataverse test;
+
+insert into dataset tsdata({"id": 1, "timestamp": datetime("-1987-11-19T23:49:23.938")})
+insert into dataset tsdata({"id": 2, "timestamp": datetime("-1987-11-20T00:27:13.432")})
+insert into dataset tsdata({"id": 3, "timestamp": datetime("-1987-11-18T18:00:00")})
+insert into dataset tsdata({"id": 4, "timestamp": datetime("19871119T234923938")})
+insert into dataset tsdata({"id": 5, "timestamp": datetime("1987-11-19T23:58:17.038")})
+insert into dataset tsdata({"id": 6, "timestamp": datetime("1987-11-19T23:30:00")})
+insert into dataset tsdata({"id": 7, "timestamp": datetime("1987-11-19T23:22:38")})
+insert into dataset tsdata({"id": 8, "timestamp": datetime("1988-01-21T17:28:13.900")})
+insert into dataset tsdata({"id": 9, "timestamp": datetime("-1987-11-19T23:49:23.938")})
+insert into dataset tsdata({"id": 10, "timestamp": datetime("-0987-07-01T09:35:28.039")})
+insert into dataset tsdata({"id": 11, "timestamp": datetime("2012-11-19T23:49:23.938")})
+insert into dataset tsdata({"id": 12, "timestamp": datetime("2013-11-19T23:49:23.938")})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_1/interval_bin_gby_1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_1/interval_bin_gby_1.3.query.aql
new file mode 100644
index 0000000..61a4c5b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_bin_gby_1/interval_bin_gby_1.3.query.aql
@@ -0,0 +1,11 @@
+/**
+ * Interval_bin_gby test case: test the group-by using interval-bin function
+ * Expected result: success
+ **/
+
+use dataverse test;
+
+for $i in dataset tsdata
+group by $d := interval-bin(get-time-from-datetime($i.timestamp), time("00:00:00.000Z"), day-time-duration("PT10M")) with $i
+order by get-interval-start($d)
+return { "tbin": $d, "count": count($i)}
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.1.ddl.aql
new file mode 100644
index 0000000..b899dc2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.1.ddl.aql
@@ -0,0 +1,7 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
+
+drop dataverse test if exists;
+create dataverse test;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.2.update.aql
new file mode 100644
index 0000000..ec16dd3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.2.update.aql
@@ -0,0 +1,4 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.3.query.aql
new file mode 100644
index 0000000..9f9f0d2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_01/parse_01.3.query.aql
@@ -0,0 +1,26 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
+
+use dataverse test;
+
+{ "date1": parse-date("2013-8-23", "YY-M-D"),
+ "date2": parse-date("Aug 12 -12", "MMM D Y"),
+ "date3": parse-date("-1234-01-01", "YYYY-MM-DD"),
+ "date4": parse-date("09/11/-1980", "D/M/Y"),
+ "date5": parse-date("09/11/-1990", "YY-M-D|MMM D Y|D/M/Y"),
+ "date6": parse-date("Mon Aug 19 2013", "W MMM D Y"),
+ "data7": parse-date("SKIPMEPLEASE Mon Aug SKIPME1ALSO 19 2013", "O W MMM O D Y"),
+ "time1": parse-time("8:23:49", "h:m:s"),
+ "time2": parse-time("8.19.23:32", "h.m.s:nn"),
+ "time3": parse-time("08.19.23:32 pm", "h.m.s:nn a"),
+ "time4": parse-time("6:30:40.948 pm PST", "h:mm:ss.nnn a z"),
+ "time5": parse-time("6:30:40.948 pm PST", "h:m:s|h.m.s:nn|h.m.s:nn a|h:mm:ss.nnn a z"),
+ "datetime1": parse-datetime("Dec 30 -1203 3:48:27 PM", "MMM DD YYYY h:m:s a"),
+ "datetime2": parse-datetime("12/30/-1203 03:48:27.392 PM Asia/Shanghai", "MM/DD/YYY hh:mm:ss.nnn a z"),
+ "datetime3": parse-datetime("1723-12-03T23:59:23.392Z", "YYYY-MM-DDThh:mm:ss.nnnz"),
+ "datetime4": parse-datetime("1723-12-03T23:59:23.392-04:00", "YYYY-MM-DDThh:mm:ss.nnnz"),
+ "datetime5": parse-datetime("1723-12-03T23:59:23.392-04:00", "MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DDThh:mm:ss.nnnz"),
+ "datetime6": parse-datetime("1970-01-01 Thu 23:59:23.392-04:00", "MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DD W hh:mm:ss.nnnz"),
+ "datetime7": parse-datetime("1723-12-03 What3v3r STRINGHERE 23:59:23.392-04:00", "MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DD O O hh:mm:ss.nnnz") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.1.ddl.aql
new file mode 100644
index 0000000..b899dc2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.1.ddl.aql
@@ -0,0 +1,7 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
+
+drop dataverse test if exists;
+create dataverse test;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.2.update.aql
new file mode 100644
index 0000000..ec16dd3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.2.update.aql
@@ -0,0 +1,4 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.3.query.aql
new file mode 100644
index 0000000..7617d5f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/parse_02/parse_02.3.query.aql
@@ -0,0 +1,20 @@
+/**
+ * Test case for parsing temporal strings with format strings
+ * Expected Result: Success
+ **/
+
+use dataverse test;
+
+let $date := date("-0123-01-30")
+let $time := time("08:07:29.030Z")
+let $datetime := datetime("0137-12-31T23:59:59.999+08:00")
+return {
+ "date-string-1": print-date($date, "YY/M/D"),
+ "date-string-2": print-date($date, "MMM DD, YYYY"),
+ "date-string-3": print-date($date, "YYYY/MM/DD"),
+ "time-string-1": print-time($time, "h.m.s.nn a z"),
+ "time-string-2": print-time($time, "hh.mm.ss.nnn a z"),
+ "datetime-string-1": print-datetime($datetime, "MMM DD h:m:s.nnn a YY z"),
+ "datetime-string-2": print-datetime($datetime, "YYYY/MMM/DD h:m:s.nnnz a"),
+ "datetime-string-3": print-datetime($datetime, "YYYY-MM-DDThh:mm:ss.nnnz")
+ }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/time_functions/time_functions.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/time_functions/time_functions.3.query.aql
index d2befcc..09e1aa9 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/time_functions/time_functions.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/time_functions/time_functions.3.query.aql
@@ -8,18 +8,18 @@
let $null2 := get-time-from-datetime(null)
let $dt2 := datetime("2012-10-11T02:30:23+03:00")
let $t3 := get-time-from-datetime($dt2)
-let $dr1 := duration("-PT30H")
-let $t4 := add-time-duration($t1, $dr1)
-let $null3 := add-time-duration(null, $dr1)
-let $null4 := add-time-duration($t1, null)
-let $c1 := $t1 = add-time-duration($t4, subtract-time($t1, $t4))
-let $dr2 := duration("PT36M")
-let $t5 := add-time-duration($t2, $dr2)
-let $c2 := $t2 = add-time-duration($t5, subtract-time($t2, $t5))
-let $dr3 := subtract-time($t5, $t2)
-let $dr4 := subtract-time($t4, $t1)
-let $null5 := subtract-time(null, $t1)
-let $null6 := subtract-time($t4, null)
+let $dr1 := day-time-duration("-PT30H")
+let $t4 := $t1 + $dr1
+let $null3 := null + $dr1
+let $null4 := $t1 + null
+let $c1 := $t1 = ($t1 - $t4) + $t4
+let $dr2 := day-time-duration("PT36M")
+let $t5 := $t2 + $dr2
+let $c2 := $t2 = $t5 + ($t2 - $t5)
+let $dr3 := $t5 - $t2
+let $dr4 := $t4 - $t1
+let $null5 := null - $t1
+let $null6 := $t4 - null
let $ct := current-time()
let $cd := current-date()
let $cdt := current-datetime()
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.update.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q2_minimum_cost_supplier/q2_minimum_cost_supplier.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q2_minimum_cost_supplier/q2_minimum_cost_supplier.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q2_minimum_cost_supplier/q2_minimum_cost_supplier.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.2.update.aql
similarity index 80%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q2_minimum_cost_supplier/q2_minimum_cost_supplier.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q2_minimum_cost_supplier/q2_minimum_cost_supplier.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q2_minimum_cost_supplier/q2_minimum_cost_supplier.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q2_minimum_cost_supplier/q2_minimum_cost_supplier.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q3_shipping_priority_nt/q3_shipping_priority_nt.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q3_shipping_priority_nt/q3_shipping_priority_nt.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q3_shipping_priority_nt/q3_shipping_priority_nt.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.2.update.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q3_shipping_priority_nt/q3_shipping_priority_nt.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q3_shipping_priority_nt/q3_shipping_priority_nt.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q3_shipping_priority_nt/q3_shipping_priority_nt.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q4_order_priority/q4_order_priority.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q4_order_priority/q4_order_priority.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q4_order_priority/q4_order_priority.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.2.update.aql
similarity index 80%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q4_order_priority/q4_order_priority.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q4_order_priority/q4_order_priority.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q4_order_priority/q4_order_priority.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q4_order_priority/q4_order_priority.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q04_order_priority/q04_order_priority.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q5_local_supplier_volume/q5_local_supplier_volume.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q5_local_supplier_volume/q5_local_supplier_volume.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q5_local_supplier_volume/q5_local_supplier_volume.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.2.update.aql
similarity index 80%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q5_local_supplier_volume/q5_local_supplier_volume.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q5_local_supplier_volume/q5_local_supplier_volume.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q5_local_supplier_volume/q5_local_supplier_volume.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q5_local_supplier_volume/q5_local_supplier_volume.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q05_local_supplier_volume/q05_local_supplier_volume.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.2.update.aql
similarity index 80%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q7_volume_shipping/q7_volume_shipping.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q7_volume_shipping/q7_volume_shipping.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q7_volume_shipping/q7_volume_shipping.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.2.update.aql
similarity index 80%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q7_volume_shipping/q7_volume_shipping.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q7_volume_shipping/q7_volume_shipping.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q7_volume_shipping/q7_volume_shipping.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q7_volume_shipping/q7_volume_shipping.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q07_volume_shipping/q07_volume_shipping.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q8_national_market_share/q8_national_market_share.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q8_national_market_share/q8_national_market_share.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q8_national_market_share/q8_national_market_share.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.2.update.aql
similarity index 80%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q8_national_market_share/q8_national_market_share.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.2.update.aql
index b23ff7f..ff66e4b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q8_national_market_share/q8_national_market_share.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.2.update.aql
@@ -2,32 +2,32 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q8_national_market_share/q8_national_market_share.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q8_national_market_share/q8_national_market_share.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q08_national_market_share/q08_national_market_share.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.1.ddl.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.1.ddl.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.2.update.aql
similarity index 80%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.2.update.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item/q10_returned_item.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item/q10_returned_item.2.update.aql
index 0adc8e1..22a7590 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item/q10_returned_item.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item/q10_returned_item.2.update.aql
@@ -6,29 +6,29 @@
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item_int64/q10_returned_item_int64.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item_int64/q10_returned_item_int64.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item_int64/q10_returned_item_int64.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q10_returned_item_int64/q10_returned_item_int64.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q11_important_stock/q11_important_stock.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q11_important_stock/q11_important_stock.2.update.aql
index b23ff7f..ff66e4b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q11_important_stock/q11_important_stock.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q11_important_stock/q11_important_stock.2.update.aql
@@ -2,32 +2,32 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q12_shipping/q12_shipping.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q12_shipping/q12_shipping.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q12_shipping/q12_shipping.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q12_shipping/q12_shipping.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q14_promotion_effect/q14_promotion_effect.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q14_promotion_effect/q14_promotion_effect.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q14_promotion_effect/q14_promotion_effect.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q14_promotion_effect/q14_promotion_effect.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q15_top_supplier/q15_top_supplier.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q18_large_volume_customer/q18_large_volume_customer.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q18_large_volume_customer/q18_large_volume_customer.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q18_large_volume_customer/q18_large_volume_customer.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q18_large_volume_customer/q18_large_volume_customer.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q20_potential_part_promotion/q20_potential_part_promotion.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q20_potential_part_promotion/q20_potential_part_promotion.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q20_potential_part_promotion/q20_potential_part_promotion.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q20_potential_part_promotion/q20_potential_part_promotion.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.2.update.aql
index 0adc8e1..0084309 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.2.update.aql
@@ -2,33 +2,33 @@
load dataset LineItem
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Orders
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Supplier
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/supplier.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Region
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/region.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Nation
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/nation.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Part
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/part.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Partsupp
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/partsupp.tbl"),("format"="delimited-text"),("delimiter"="|"));
load dataset Customer
using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+(("path"="nc1://data/tpch0.001/customer.tbl"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/types/record01/record01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/types/record01/record01.1.ddl.aql
new file mode 100644
index 0000000..faae040
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/types/record01/record01.1.ddl.aql
@@ -0,0 +1,5 @@
+drop dataverse local if exists;
+create dataverse local;
+use dataverse local;
+create type ttype as { "id" : int32 } ;
+create dataset dset (ttype) primary key id;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/aggregate/issue531_string_min_max/issue531_string_min_max.1.adm b/asterix-app/src/test/resources/runtimets/results/aggregate/issue531_string_min_max/issue531_string_min_max.1.adm
new file mode 100644
index 0000000..9f575f1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/aggregate/issue531_string_min_max/issue531_string_min_max.1.adm
@@ -0,0 +1 @@
+{ "min": "Alex", "max": "Susan" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.adm
new file mode 100644
index 0000000..bd9b76a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.adm
@@ -0,0 +1,240 @@
+{ "id": 101, "fname": "Javier", "lname": "Makuch", "age": 28, "dept": "IT" }
+{ "id": 110, "fname": "Allan", "lname": "Piland", "age": 29, "dept": "HR" }
+{ "id": 112, "fname": "Pearlie", "lname": "Aumann", "age": 31, "dept": "Payroll" }
+{ "id": 113, "fname": "Chandra", "lname": "Hase", "age": 34, "dept": "Sales" }
+{ "id": 114, "fname": "Christian", "lname": "Convery", "age": 28, "dept": "HR" }
+{ "id": 115, "fname": "Panther", "lname": "Ritch", "age": 26, "dept": "IT" }
+{ "id": 116, "fname": "Ted", "lname": "Elsea", "age": 26, "dept": "IT" }
+{ "id": 117, "fname": "Tabatha", "lname": "Bladen", "age": 25, "dept": "HR" }
+{ "id": 118, "fname": "Clayton", "lname": "Oltman", "age": 42, "dept": "Sales" }
+{ "id": 119, "fname": "Sharron", "lname": "Darwin", "age": 32, "dept": "Payroll" }
+{ "id": 210, "fname": "Clayton", "lname": "Durgin", "age": 52, "dept": "HR" }
+{ "id": 212, "fname": "Emilia", "lname": "Chenail", "age": 26, "dept": "Sales" }
+{ "id": 213, "fname": "Kenya", "lname": "Almquist", "age": 43, "dept": "Payroll" }
+{ "id": 214, "fname": "Alejandra", "lname": "Lacefield", "age": 41, "dept": "HR" }
+{ "id": 215, "fname": "Karina", "lname": "Michelsen", "age": 46, "dept": "IT" }
+{ "id": 216, "fname": "Katy", "lname": "Delillo", "age": 36, "dept": "IT" }
+{ "id": 217, "fname": "Benita", "lname": "Kleist", "age": 37, "dept": "HR" }
+{ "id": 218, "fname": "Earlene", "lname": "Paluch", "age": 31, "dept": "IT" }
+{ "id": 219, "fname": "Kurt", "lname": "Petermann", "age": 27, "dept": "Payroll" }
+{ "id": 299, "fname": "Julio", "lname": "Iorio", "age": 37, "dept": "IT" }
+{ "id": 363, "fname": "Cody", "lname": "Rodreguez", "age": 26, "dept": "IT" }
+{ "id": 404, "fname": "Emilia", "lname": "Square", "age": 32, "dept": "IT" }
+{ "id": 414, "fname": "Mathew", "lname": "Fuschetto", "age": 34, "dept": "HR" }
+{ "id": 424, "fname": "Allyson", "lname": "Remus", "age": 32, "dept": "IT" }
+{ "id": 434, "fname": "Earlene", "lname": "Linebarger", "age": 26, "dept": "Payroll" }
+{ "id": 444, "fname": "Clinton", "lname": "Sick", "age": 29, "dept": "IT" }
+{ "id": 454, "fname": "Ted", "lname": "Caba", "age": 28, "dept": "HR" }
+{ "id": 463, "fname": "Marcie", "lname": "States", "age": 28, "dept": "IT" }
+{ "id": 464, "fname": "Fernando", "lname": "Engelke", "age": 39, "dept": "IT" }
+{ "id": 474, "fname": "Mathew", "lname": "Courchesne", "age": 31, "dept": "IT" }
+{ "id": 484, "fname": "Cody", "lname": "Vinyard", "age": 36, "dept": "Payroll" }
+{ "id": 494, "fname": "Benita", "lname": "Fravel", "age": 33, "dept": "Sales" }
+{ "id": 504, "fname": "Erik", "lname": "Dobek", "age": 29, "dept": "IT" }
+{ "id": 514, "fname": "Julio", "lname": "Ruben", "age": 41, "dept": "IT" }
+{ "id": 524, "fname": "Benita", "lname": "Maltos", "age": 33, "dept": "IT" }
+{ "id": 534, "fname": "Kurt", "lname": "Biscoe", "age": 36, "dept": "HR" }
+{ "id": 538, "fname": "Milagros", "lname": "Forkey", "age": 34, "dept": "Sales" }
+{ "id": 544, "fname": "Loraine", "lname": "Housel", "age": 30, "dept": "Sales" }
+{ "id": 554, "fname": "Jamie", "lname": "Rachal", "age": 30, "dept": "IT" }
+{ "id": 564, "fname": "Liza", "lname": "Fredenburg", "age": 37, "dept": "IT" }
+{ "id": 574, "fname": "Ericka", "lname": "Feldmann", "age": 29, "dept": "Sales" }
+{ "id": 584, "fname": "Dollie", "lname": "Dattilo", "age": 32, "dept": "Payroll" }
+{ "id": 589, "fname": "Lorrie", "lname": "Sharon", "age": 27, "dept": "IT" }
+{ "id": 594, "fname": "Roxie", "lname": "Houghtaling", "age": 40, "dept": "Payroll" }
+{ "id": 601, "fname": "Neil", "lname": "Deforge", "age": 26, "dept": "HR" }
+{ "id": 611, "fname": "Earlene", "lname": "Marcy", "age": 32, "dept": "IT" }
+{ "id": 621, "fname": "Erik", "lname": "Lechuga", "age": 42, "dept": "Payroll" }
+{ "id": 631, "fname": "Tyrone", "lname": "Holtzclaw", "age": 34, "dept": "Sales" }
+{ "id": 641, "fname": "Lance", "lname": "Hankey", "age": 35, "dept": "Sales" }
+{ "id": 651, "fname": "Mallory", "lname": "Gladding", "age": 31, "dept": "HR" }
+{ "id": 661, "fname": "Tia", "lname": "Braaten", "age": 40, "dept": "IT" }
+{ "id": 671, "fname": "Julio", "lname": "Vanpatten", "age": 30, "dept": "Payroll" }
+{ "id": 681, "fname": "Max", "lname": "Teachout", "age": 34, "dept": "IT" }
+{ "id": 691, "fname": "Karina", "lname": "Wingerter", "age": 31, "dept": "IT" }
+{ "id": 711, "fname": "Hugh", "lname": "Lema", "age": 25, "dept": "HR" }
+{ "id": 721, "fname": "Schwan", "lname": "Phil", "age": 34, "dept": "Payroll" }
+{ "id": 732, "fname": "Noemi", "lname": "Eacret", "age": 56, "dept": "HR" }
+{ "id": 741, "fname": "Julio", "lname": "Mattocks", "age": 38, "dept": "Sales" }
+{ "id": 751, "fname": "Lance", "lname": "Kottke", "age": 34, "dept": "IT" }
+{ "id": 761, "fname": "Kurt", "lname": "Liz", "age": 32, "dept": "HR" }
+{ "id": 771, "fname": "Neva", "lname": "Barbeau", "age": 45, "dept": "Sales" }
+{ "id": 781, "fname": "Karina", "lname": "Tuthill", "age": 46, "dept": "Payroll" }
+{ "id": 791, "fname": "Maricela", "lname": "Cambron", "age": 36, "dept": "IT" }
+{ "id": 809, "fname": "Clayton", "lname": "Delany", "age": 23, "dept": "IT" }
+{ "id": 811, "fname": "Kubik", "lname": "Kuhn", "age": 27, "dept": "HR" }
+{ "id": 821, "fname": "Allan", "lname": "Tomes", "age": 29, "dept": "Payroll" }
+{ "id": 831, "fname": "Lonnie", "lname": "Aller", "age": 33, "dept": "Sales" }
+{ "id": 841, "fname": "Neil", "lname": "Hurrell", "age": 26, "dept": "IT" }
+{ "id": 851, "fname": "Clayton", "lname": "Engles", "age": 41, "dept": "HR" }
+{ "id": 861, "fname": "Javier", "lname": "Gabrielson", "age": 39, "dept": "Payroll" }
+{ "id": 871, "fname": "Allan", "lname": "Alejandre", "age": 48, "dept": "IT" }
+{ "id": 881, "fname": "Julio", "lname": "Isa", "age": 38, "dept": "Sales" }
+{ "id": 891, "fname": "Roslyn", "lname": "Simmerman", "age": 31, "dept": "IT" }
+{ "id": 915, "fname": "Starner", "lname": "Stuart", "age": 25, "dept": "Sales" }
+{ "id": 925, "fname": "Sofia", "lname": "Cuff", "age": 30, "dept": "HR" }
+{ "id": 935, "fname": "Milagros", "lname": "Murguia", "age": 31, "dept": "IT" }
+{ "id": 945, "fname": "Margery", "lname": "Haldeman", "age": 32, "dept": "IT" }
+{ "id": 955, "fname": "Max", "lname": "Mell", "age": 33, "dept": "HR" }
+{ "id": 965, "fname": "Micco", "lname": "Mercy", "age": 31, "dept": "Payroll" }
+{ "id": 975, "fname": "Clare", "lname": "Vangieson", "age": 34, "dept": "IT" }
+{ "id": 985, "fname": "Elnora", "lname": "Dimauro", "age": 35, "dept": "Sales" }
+{ "id": 995, "fname": "Pearlie", "lname": "Kocian", "age": 38, "dept": "HR" }
+{ "id": 1007, "fname": "Yingyi", "lname": "Bu", "age": 27, "dept": "IT" }
+{ "id": 1263, "fname": "Tania", "lname": "Loffredo", "age": 25, "dept": "IT" }
+{ "id": 1410, "fname": "Clinton", "lname": "Fredricks", "age": 34, "dept": "IT" }
+{ "id": 1411, "fname": "Lance", "lname": "Farquhar", "age": 32, "dept": "HR" }
+{ "id": 1412, "fname": "Tabatha", "lname": "Crisler", "age": 33, "dept": "IT" }
+{ "id": 1413, "fname": "Max", "lname": "Durney", "age": 29, "dept": "IT" }
+{ "id": 1414, "fname": "Carmella", "lname": "Strauser", "age": 30, "dept": "Payroll" }
+{ "id": 1415, "fname": "Kelly", "lname": "Carrales", "age": 40, "dept": "IT" }
+{ "id": 1416, "fname": "Guy", "lname": "Merten", "age": 29, "dept": "Sales" }
+{ "id": 1417, "fname": "Noreen", "lname": "Ruhland", "age": 29, "dept": "IT" }
+{ "id": 1418, "fname": "Julio", "lname": "Damore", "age": 27, "dept": "Sales" }
+{ "id": 1419, "fname": "Selena", "lname": "Truby", "age": 25, "dept": "HR" }
+{ "id": 1420, "fname": "Alejandra", "lname": "Commons", "age": 30, "dept": "Sales" }
+{ "id": 1421, "fname": "Allyson", "lname": "Balk", "age": 30, "dept": "IT" }
+{ "id": 1422, "fname": "Nelson", "lname": "Byun", "age": 40, "dept": "Sales" }
+{ "id": 1423, "fname": "Christian", "lname": "Reidhead", "age": 40, "dept": "IT" }
+{ "id": 1424, "fname": "Pearlie", "lname": "Hopkin", "age": 48, "dept": "Payroll" }
+{ "id": 1425, "fname": "Nelson", "lname": "Wohlers", "age": 41, "dept": "HR" }
+{ "id": 1426, "fname": "Marcie", "lname": "Rasnake", "age": 42, "dept": "Sales" }
+{ "id": 1427, "fname": "Hugh", "lname": "Marshburn", "age": 43, "dept": "Payroll" }
+{ "id": 1428, "fname": "Mathew", "lname": "Marasco", "age": 45, "dept": "Sales" }
+{ "id": 1429, "fname": "Kurt", "lname": "Veres", "age": 32, "dept": "IT" }
+{ "id": 1430, "fname": "Julio", "lname": "Barkett", "age": 39, "dept": "Sales" }
+{ "id": 1863, "fname": "Darren", "lname": "Thorington", "age": 32, "dept": "Sales" }
+{ "id": 1999, "fname": "Susan", "lname": "Malaika", "age": 42, "dept": "HR" }
+{ "id": 2333, "fname": "Chen", "lname": "Li", "age": 42, "dept": "HR" }
+{ "id": 2963, "fname": "Neil", "lname": "Gunnerson", "age": 34, "dept": "IT" }
+{ "id": 3563, "fname": "Hazeltine", "lname": "Susan", "age": 29, "dept": "Sales" }
+{ "id": 3666, "fname": "Young Seok", "lname": "Kim", "age": 35, "dept": "Payroll" }
+{ "id": 4727, "fname": "Michael", "lname": "Carey", "age": 50, "dept": "Payroll" }
+{ "id": 5438, "fname": "Lakisha", "lname": "Quashie", "age": 29, "dept": "HR" }
+{ "id": 7444, "fname": "Sharad", "lname": "Mehrotra", "age": 42, "dept": "Sales" }
+{ "id": 7663, "fname": "Annabelle", "lname": "Nimmo", "age": 30, "dept": "Payroll" }
+{ "id": 8301, "fname": "Earlene", "lname": "Wallick", "age": 26, "dept": "HR" }
+{ "id": 8338, "fname": "Julio", "lname": "Bosket", "age": 28, "dept": "Payroll" }
+{ "id": 9555, "fname": "Tony", "lname": "Givargis", "age": 40, "dept": "Sales" }
+{ "id": 9763, "fname": "Ted", "lname": "Saini", "age": 31, "dept": "IT" }
+{ "id": 9941, "fname": "Khurram Faraaz", "lname": "Mohammed", "age": 30, "dept": "HR" }
+{ "id": 10101, "fname": "Javier", "lname": "Makuch", "age": 28, "dept": "IT" }
+{ "id": 10110, "fname": "Allan", "lname": "Piland", "age": 29, "dept": "HR" }
+{ "id": 10112, "fname": "Pearlie", "lname": "Aumann", "age": 31, "dept": "Payroll" }
+{ "id": 10113, "fname": "Chandra", "lname": "Hase", "age": 34, "dept": "Sales" }
+{ "id": 10114, "fname": "Christian", "lname": "Convery", "age": 28, "dept": "HR" }
+{ "id": 10115, "fname": "Panther", "lname": "Ritch", "age": 26, "dept": "IT" }
+{ "id": 10116, "fname": "Ted", "lname": "Elsea", "age": 26, "dept": "IT" }
+{ "id": 10117, "fname": "Tabatha", "lname": "Bladen", "age": 25, "dept": "HR" }
+{ "id": 10118, "fname": "Clayton", "lname": "Oltman", "age": 42, "dept": "Sales" }
+{ "id": 10119, "fname": "Sharron", "lname": "Darwin", "age": 32, "dept": "Payroll" }
+{ "id": 10210, "fname": "Clayton", "lname": "Durgin", "age": 52, "dept": "HR" }
+{ "id": 10212, "fname": "Emilia", "lname": "Chenail", "age": 26, "dept": "Sales" }
+{ "id": 10213, "fname": "Kenya", "lname": "Almquist", "age": 43, "dept": "Payroll" }
+{ "id": 10214, "fname": "Alejandra", "lname": "Lacefield", "age": 41, "dept": "HR" }
+{ "id": 10215, "fname": "Karina", "lname": "Michelsen", "age": 46, "dept": "IT" }
+{ "id": 10216, "fname": "Katy", "lname": "Delillo", "age": 36, "dept": "IT" }
+{ "id": 10217, "fname": "Benita", "lname": "Kleist", "age": 37, "dept": "HR" }
+{ "id": 10218, "fname": "Earlene", "lname": "Paluch", "age": 31, "dept": "IT" }
+{ "id": 10219, "fname": "Kurt", "lname": "Petermann", "age": 27, "dept": "Payroll" }
+{ "id": 10299, "fname": "Julio", "lname": "Iorio", "age": 37, "dept": "IT" }
+{ "id": 10363, "fname": "Cody", "lname": "Rodreguez", "age": 26, "dept": "IT" }
+{ "id": 10404, "fname": "Emilia", "lname": "Square", "age": 32, "dept": "IT" }
+{ "id": 10414, "fname": "Mathew", "lname": "Fuschetto", "age": 34, "dept": "HR" }
+{ "id": 10424, "fname": "Allyson", "lname": "Remus", "age": 32, "dept": "IT" }
+{ "id": 10434, "fname": "Earlene", "lname": "Linebarger", "age": 26, "dept": "Payroll" }
+{ "id": 10444, "fname": "Clinton", "lname": "Sick", "age": 29, "dept": "IT" }
+{ "id": 10454, "fname": "Ted", "lname": "Caba", "age": 28, "dept": "HR" }
+{ "id": 10463, "fname": "Marcie", "lname": "States", "age": 28, "dept": "IT" }
+{ "id": 10464, "fname": "Fernando", "lname": "Engelke", "age": 39, "dept": "IT" }
+{ "id": 10474, "fname": "Mathew", "lname": "Courchesne", "age": 31, "dept": "IT" }
+{ "id": 10484, "fname": "Cody", "lname": "Vinyard", "age": 36, "dept": "Payroll" }
+{ "id": 10494, "fname": "Benita", "lname": "Fravel", "age": 33, "dept": "Sales" }
+{ "id": 10504, "fname": "Erik", "lname": "Dobek", "age": 29, "dept": "IT" }
+{ "id": 10514, "fname": "Julio", "lname": "Ruben", "age": 41, "dept": "IT" }
+{ "id": 10524, "fname": "Benita", "lname": "Maltos", "age": 33, "dept": "IT" }
+{ "id": 10534, "fname": "Kurt", "lname": "Biscoe", "age": 36, "dept": "HR" }
+{ "id": 10538, "fname": "Milagros", "lname": "Forkey", "age": 34, "dept": "Sales" }
+{ "id": 10544, "fname": "Loraine", "lname": "Housel", "age": 30, "dept": "Sales" }
+{ "id": 10554, "fname": "Jamie", "lname": "Rachal", "age": 30, "dept": "IT" }
+{ "id": 10564, "fname": "Liza", "lname": "Fredenburg", "age": 37, "dept": "IT" }
+{ "id": 10574, "fname": "Ericka", "lname": "Feldmann", "age": 29, "dept": "Sales" }
+{ "id": 10584, "fname": "Dollie", "lname": "Dattilo", "age": 32, "dept": "Payroll" }
+{ "id": 10589, "fname": "Lorrie", "lname": "Sharon", "age": 27, "dept": "IT" }
+{ "id": 10594, "fname": "Roxie", "lname": "Houghtaling", "age": 40, "dept": "Payroll" }
+{ "id": 10601, "fname": "Neil", "lname": "Deforge", "age": 26, "dept": "HR" }
+{ "id": 10611, "fname": "Earlene", "lname": "Marcy", "age": 32, "dept": "IT" }
+{ "id": 10621, "fname": "Erik", "lname": "Lechuga", "age": 42, "dept": "Payroll" }
+{ "id": 10631, "fname": "Tyrone", "lname": "Holtzclaw", "age": 34, "dept": "Sales" }
+{ "id": 10641, "fname": "Lance", "lname": "Hankey", "age": 35, "dept": "Sales" }
+{ "id": 10651, "fname": "Mallory", "lname": "Gladding", "age": 31, "dept": "HR" }
+{ "id": 10661, "fname": "Tia", "lname": "Braaten", "age": 40, "dept": "IT" }
+{ "id": 10671, "fname": "Julio", "lname": "Vanpatten", "age": 30, "dept": "Payroll" }
+{ "id": 10681, "fname": "Max", "lname": "Teachout", "age": 34, "dept": "IT" }
+{ "id": 10691, "fname": "Karina", "lname": "Wingerter", "age": 31, "dept": "IT" }
+{ "id": 10711, "fname": "Hugh", "lname": "Lema", "age": 25, "dept": "HR" }
+{ "id": 10721, "fname": "Schwan", "lname": "Phil", "age": 34, "dept": "Payroll" }
+{ "id": 10732, "fname": "Noemi", "lname": "Eacret", "age": 56, "dept": "HR" }
+{ "id": 10741, "fname": "Julio", "lname": "Mattocks", "age": 38, "dept": "Sales" }
+{ "id": 10751, "fname": "Lance", "lname": "Kottke", "age": 34, "dept": "IT" }
+{ "id": 10761, "fname": "Kurt", "lname": "Liz", "age": 32, "dept": "HR" }
+{ "id": 10771, "fname": "Neva", "lname": "Barbeau", "age": 45, "dept": "Sales" }
+{ "id": 10781, "fname": "Karina", "lname": "Tuthill", "age": 46, "dept": "Payroll" }
+{ "id": 10791, "fname": "Maricela", "lname": "Cambron", "age": 36, "dept": "IT" }
+{ "id": 10809, "fname": "Clayton", "lname": "Delany", "age": 23, "dept": "IT" }
+{ "id": 10811, "fname": "Kubik", "lname": "Kuhn", "age": 27, "dept": "HR" }
+{ "id": 10821, "fname": "Allan", "lname": "Tomes", "age": 29, "dept": "Payroll" }
+{ "id": 10831, "fname": "Lonnie", "lname": "Aller", "age": 33, "dept": "Sales" }
+{ "id": 10841, "fname": "Neil", "lname": "Hurrell", "age": 26, "dept": "IT" }
+{ "id": 10851, "fname": "Clayton", "lname": "Engles", "age": 41, "dept": "HR" }
+{ "id": 10861, "fname": "Javier", "lname": "Gabrielson", "age": 39, "dept": "Payroll" }
+{ "id": 10871, "fname": "Allan", "lname": "Alejandre", "age": 48, "dept": "IT" }
+{ "id": 10881, "fname": "Julio", "lname": "Isa", "age": 38, "dept": "Sales" }
+{ "id": 10891, "fname": "Roslyn", "lname": "Simmerman", "age": 31, "dept": "IT" }
+{ "id": 10915, "fname": "Starner", "lname": "Stuart", "age": 25, "dept": "Sales" }
+{ "id": 10925, "fname": "Sofia", "lname": "Cuff", "age": 30, "dept": "HR" }
+{ "id": 10935, "fname": "Milagros", "lname": "Murguia", "age": 31, "dept": "IT" }
+{ "id": 10945, "fname": "Margery", "lname": "Haldeman", "age": 32, "dept": "IT" }
+{ "id": 10955, "fname": "Max", "lname": "Mell", "age": 33, "dept": "HR" }
+{ "id": 10965, "fname": "Micco", "lname": "Mercy", "age": 31, "dept": "Payroll" }
+{ "id": 10975, "fname": "Clare", "lname": "Vangieson", "age": 34, "dept": "IT" }
+{ "id": 10985, "fname": "Elnora", "lname": "Dimauro", "age": 35, "dept": "Sales" }
+{ "id": 10995, "fname": "Pearlie", "lname": "Kocian", "age": 38, "dept": "HR" }
+{ "id": 11007, "fname": "Yingyi", "lname": "Bu", "age": 27, "dept": "IT" }
+{ "id": 11263, "fname": "Tania", "lname": "Loffredo", "age": 25, "dept": "IT" }
+{ "id": 11410, "fname": "Clinton", "lname": "Fredricks", "age": 34, "dept": "IT" }
+{ "id": 11411, "fname": "Lance", "lname": "Farquhar", "age": 32, "dept": "HR" }
+{ "id": 11412, "fname": "Tabatha", "lname": "Crisler", "age": 33, "dept": "IT" }
+{ "id": 11413, "fname": "Max", "lname": "Durney", "age": 29, "dept": "IT" }
+{ "id": 11414, "fname": "Carmella", "lname": "Strauser", "age": 30, "dept": "Payroll" }
+{ "id": 11415, "fname": "Kelly", "lname": "Carrales", "age": 40, "dept": "IT" }
+{ "id": 11416, "fname": "Guy", "lname": "Merten", "age": 29, "dept": "Sales" }
+{ "id": 11417, "fname": "Noreen", "lname": "Ruhland", "age": 29, "dept": "IT" }
+{ "id": 11418, "fname": "Julio", "lname": "Damore", "age": 27, "dept": "Sales" }
+{ "id": 11419, "fname": "Selena", "lname": "Truby", "age": 25, "dept": "HR" }
+{ "id": 11420, "fname": "Alejandra", "lname": "Commons", "age": 30, "dept": "Sales" }
+{ "id": 11421, "fname": "Allyson", "lname": "Balk", "age": 30, "dept": "IT" }
+{ "id": 11422, "fname": "Nelson", "lname": "Byun", "age": 40, "dept": "Sales" }
+{ "id": 11423, "fname": "Christian", "lname": "Reidhead", "age": 40, "dept": "IT" }
+{ "id": 11424, "fname": "Pearlie", "lname": "Hopkin", "age": 48, "dept": "Payroll" }
+{ "id": 11425, "fname": "Nelson", "lname": "Wohlers", "age": 41, "dept": "HR" }
+{ "id": 11426, "fname": "Marcie", "lname": "Rasnake", "age": 42, "dept": "Sales" }
+{ "id": 11427, "fname": "Hugh", "lname": "Marshburn", "age": 43, "dept": "Payroll" }
+{ "id": 11428, "fname": "Mathew", "lname": "Marasco", "age": 45, "dept": "Sales" }
+{ "id": 11429, "fname": "Kurt", "lname": "Veres", "age": 32, "dept": "IT" }
+{ "id": 11430, "fname": "Julio", "lname": "Barkett", "age": 39, "dept": "Sales" }
+{ "id": 11863, "fname": "Darren", "lname": "Thorington", "age": 32, "dept": "Sales" }
+{ "id": 11999, "fname": "Susan", "lname": "Malaika", "age": 42, "dept": "HR" }
+{ "id": 12333, "fname": "Chen", "lname": "Li", "age": 42, "dept": "HR" }
+{ "id": 12963, "fname": "Neil", "lname": "Gunnerson", "age": 34, "dept": "IT" }
+{ "id": 13563, "fname": "Hazeltine", "lname": "Susan", "age": 29, "dept": "Sales" }
+{ "id": 13666, "fname": "Young Seok", "lname": "Kim", "age": 35, "dept": "Payroll" }
+{ "id": 14727, "fname": "Michael", "lname": "Carey", "age": 50, "dept": "Payroll" }
+{ "id": 15438, "fname": "Lakisha", "lname": "Quashie", "age": 29, "dept": "HR" }
+{ "id": 17444, "fname": "Sharad", "lname": "Mehrotra", "age": 42, "dept": "Sales" }
+{ "id": 17663, "fname": "Annabelle", "lname": "Nimmo", "age": 30, "dept": "Payroll" }
+{ "id": 18301, "fname": "Earlene", "lname": "Wallick", "age": 26, "dept": "HR" }
+{ "id": 18338, "fname": "Julio", "lname": "Bosket", "age": 28, "dept": "Payroll" }
+{ "id": 19555, "fname": "Tony", "lname": "Givargis", "age": 40, "dept": "Sales" }
+{ "id": 19763, "fname": "Ted", "lname": "Saini", "age": 31, "dept": "IT" }
+{ "id": 19941, "fname": "Khurram Faraaz", "lname": "Mohammed", "age": 30, "dept": "HR" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.adm
new file mode 100644
index 0000000..29267a9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.adm
@@ -0,0 +1,20 @@
+{ "id": 1 }
+{ "id": 2 }
+{ "id": 3 }
+{ "id": 4 }
+{ "id": 5 }
+{ "id": 6 }
+{ "id": 7 }
+{ "id": 8 }
+{ "id": 9 }
+{ "id": 10 }
+{ "id": 11 }
+{ "id": 12 }
+{ "id": 13 }
+{ "id": 14 }
+{ "id": 15 }
+{ "id": 16 }
+{ "id": 17 }
+{ "id": 18 }
+{ "id": 19 }
+{ "id": 20 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.adm
new file mode 100644
index 0000000..73a793c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.adm
@@ -0,0 +1,15 @@
+{ "id": 1 }
+{ "id": 2 }
+{ "id": 3 }
+{ "id": 4 }
+{ "id": 5 }
+{ "id": 6 }
+{ "id": 7 }
+{ "id": 8 }
+{ "id": 9 }
+{ "id": 10 }
+{ "id": 11 }
+{ "id": 13 }
+{ "id": 15 }
+{ "id": 17 }
+{ "id": 19 }
diff --git a/asterix-app/src/test/resources/runtimets/results/failure/q1_pricing_summary_report_failure/q1_pricing_summary_report_failure.1.adm b/asterix-app/src/test/resources/runtimets/results/failure/q01_pricing_summary_report_failure/q01_pricing_summary_report_failure.1.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/failure/q1_pricing_summary_report_failure/q1_pricing_summary_report_failure.1.adm
rename to asterix-app/src/test/resources/runtimets/results/failure/q01_pricing_summary_report_failure/q01_pricing_summary_report_failure.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-aqlplus_1/dblp-aqlplus_1.1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-aqlplus_1/dblp-aqlplus_1.1.adm
index 10357e5..b79bfe0 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-aqlplus_1/dblp-aqlplus_1.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-aqlplus_1/dblp-aqlplus_1.1.adm
@@ -1,3 +1,4 @@
+{ "dblp": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "dblp2": { "id": 24, "dblpid": "books/acm/kim95/OzsuB95", "title": "Query Processing in Object-Oriented Database Systems.", "authors": "M. Tamer Özsu José A. Blakeley", "misc": "2002-01-03 146-174 1995 Modern Database Systems db/books/collections/kim95.html#OzsuB95" } }
{ "dblp": { "id": 81, "dblpid": "journals/siamcomp/AspnesW96", "title": "Randomized Consensus in Expected O(n log² n) Operations Per Processor.", "authors": "James Aspnes Orli Waarts", "misc": "2002-01-03 1024-1044 1996 25 SIAM J. Comput. 5 db/journals/siamcomp/siamcomp25.html#AspnesW96" }, "dblp2": { "id": 82, "dblpid": "conf/focs/AspnesW92", "title": "Randomized Consensus in Expected O(n log ^2 n) Operations Per Processor", "authors": "James Aspnes Orli Waarts", "misc": "2006-04-25 137-146 conf/focs/FOCS33 1992 FOCS db/conf/focs/focs92.html#AspnesW92" } }
{ "dblp": { "id": 83, "dblpid": "journals/siamcomp/Bloniarz83", "title": "A Shortest-Path Algorithm with Expected Time O(n² log n log* n).", "authors": "Peter A. Bloniarz", "misc": "2002-01-03 588-600 1983 12 SIAM J. Comput. 3 db/journals/siamcomp/siamcomp12.html#Bloniarz83" }, "dblp2": { "id": 84, "dblpid": "conf/stoc/Bloniarz80", "title": "A Shortest-Path Algorithm with Expected Time O(n^2 log n log ^* n)", "authors": "Peter A. Bloniarz", "misc": "2006-04-25 378-384 conf/stoc/STOC12 1980 STOC db/conf/stoc/stoc80.html#Bloniarz80" } }
{ "dblp": { "id": 83, "dblpid": "journals/siamcomp/Bloniarz83", "title": "A Shortest-Path Algorithm with Expected Time O(n² log n log* n).", "authors": "Peter A. Bloniarz", "misc": "2002-01-03 588-600 1983 12 SIAM J. Comput. 3 db/journals/siamcomp/siamcomp12.html#Bloniarz83" }, "dblp2": { "id": 87, "dblpid": "journals/siamcomp/MoffatT87", "title": "An All Pairs Shortest Path Algorithm with Expected Time O(n² log n).", "authors": "Alistair Moffat Tadao Takaoka", "misc": "2002-01-03 1023-1031 1987 16 SIAM J. Comput. 6 db/journals/siamcomp/siamcomp16.html#MoffatT87" } }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_1/dblp-csx-aqlplus_1.1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_1/dblp-csx-aqlplus_1.1.adm
index 9a58a6d..7848ca1 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_1/dblp-csx-aqlplus_1.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_1/dblp-csx-aqlplus_1.1.adm
@@ -1,5 +1,6 @@
{ "dblp": { "id": 1, "dblpid": "books/acm/kim95/AnnevelinkACFHK95", "title": "Object SQL - A Language for the Design and Implementation of Object Databases.", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent", "misc": "2002-01-03 42-68 1995 Modern Database Systems db/books/collections/kim95.html#AnnevelinkACFHK95" }, "csx": { "id": 1, "csxid": "oai CiteSeerXPSU 10.1.1.39.1830", "title": "Object SQL - A Language for the Design and Implementation of Object Databases", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Dan Fishman Mike Heytens William Kent", "misc": "2009-04-13 ly, a function application expression consists of two expressions a function reference (labelled func_ref in Figure 3 line 2), and an argument (labelled arg). The func_ref expression evaluates to a (generic or specific) function identifier, which may be the same as the function that the expression is a part of, thus allowing recursive function invocations. The expression labelled arg evaluates to an arbitrary object or aggregate object. The semantics of evaluating function applications was discussed in detail in section 2. For example, to set the name of a person, we evaluate the following expression FunAssign(function name.person) (p1,'John') In this example, the first expression is itself a function call, applying the function FunAssign to the function name.person (an example of a specific function reference). This returns the oid of the function that sets a person's name, which is subsequently applied to a tuple of two elements, the oid of the person and the new name (a string o... CiteSeerX ACM Press 2009-04-13 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.39.1830 http //www.tu-chemnitz.de/~igrdb/docs/OpenODB/osql.ps.gz en 10.1.1.31.2534 10.1.1.28.4658 10.1.1.44.5947 10.1.1.39.199 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "csx": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems topics covered include active database rule models and languages, rule execution semantics, and implementation issues. 1 Introduction Conventional database systems are passive they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
+{ "dblp": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "csx": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema. The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer. When a global query is submitted, it is decomposed into two types of queries [1] -- subqueries, operating on sharable data items from local databases, -- assembling queries, consisting of, CiteSeerX 2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "csx": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of different types) over different processing entities. Such applications are referred to as transactional workflows. In this chapter we discuss the specification of such workflows and the issues involved in their execution. 1 What is a Workflow? Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX 2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 93, "csxid": "oai CiteSeerXPSU 10.1.1.42.9253", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-11 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of office information systems it is costly and difficult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"office objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to office software. In order to fully exploit the approach to achieve integrated office systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt t CiteSeerX ACM Press and Addison-Wesley 2009-04-11 2007-11-22 1988 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.42.9253 ftp //ftp.iam.unibe.ch/pub/scg/Papers/integratedOfficeSystems.ps.gz en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_2/dblp-csx-aqlplus_2.1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_2/dblp-csx-aqlplus_2.1.adm
index 9a58a6d..7848ca1 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_2/dblp-csx-aqlplus_2.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_2/dblp-csx-aqlplus_2.1.adm
@@ -1,5 +1,6 @@
{ "dblp": { "id": 1, "dblpid": "books/acm/kim95/AnnevelinkACFHK95", "title": "Object SQL - A Language for the Design and Implementation of Object Databases.", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent", "misc": "2002-01-03 42-68 1995 Modern Database Systems db/books/collections/kim95.html#AnnevelinkACFHK95" }, "csx": { "id": 1, "csxid": "oai CiteSeerXPSU 10.1.1.39.1830", "title": "Object SQL - A Language for the Design and Implementation of Object Databases", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Dan Fishman Mike Heytens William Kent", "misc": "2009-04-13 ly, a function application expression consists of two expressions a function reference (labelled func_ref in Figure 3 line 2), and an argument (labelled arg). The func_ref expression evaluates to a (generic or specific) function identifier, which may be the same as the function that the expression is a part of, thus allowing recursive function invocations. The expression labelled arg evaluates to an arbitrary object or aggregate object. The semantics of evaluating function applications was discussed in detail in section 2. For example, to set the name of a person, we evaluate the following expression FunAssign(function name.person) (p1,'John') In this example, the first expression is itself a function call, applying the function FunAssign to the function name.person (an example of a specific function reference). This returns the oid of the function that sets a person's name, which is subsequently applied to a tuple of two elements, the oid of the person and the new name (a string o... CiteSeerX ACM Press 2009-04-13 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.39.1830 http //www.tu-chemnitz.de/~igrdb/docs/OpenODB/osql.ps.gz en 10.1.1.31.2534 10.1.1.28.4658 10.1.1.44.5947 10.1.1.39.199 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "csx": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems topics covered include active database rule models and languages, rule execution semantics, and implementation issues. 1 Introduction Conventional database systems are passive they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
+{ "dblp": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "csx": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema. The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer. When a global query is submitted, it is decomposed into two types of queries [1] -- subqueries, operating on sharable data items from local databases, -- assembling queries, consisting of, CiteSeerX 2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "csx": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of different types) over different processing entities. Such applications are referred to as transactional workflows. In this chapter we discuss the specification of such workflows and the issues involved in their execution. 1 What is a Workflow? Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX 2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 93, "csxid": "oai CiteSeerXPSU 10.1.1.42.9253", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-11 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of office information systems it is costly and difficult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"office objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to office software. In order to fully exploit the approach to achieve integrated office systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt t CiteSeerX ACM Press and Addison-Wesley 2009-04-11 2007-11-22 1988 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.42.9253 ftp //ftp.iam.unibe.ch/pub/scg/Papers/integratedOfficeSystems.ps.gz en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_3/dblp-csx-aqlplus_3.1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_3/dblp-csx-aqlplus_3.1.adm
index 9a58a6d..7848ca1 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_3/dblp-csx-aqlplus_3.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_3/dblp-csx-aqlplus_3.1.adm
@@ -1,5 +1,6 @@
{ "dblp": { "id": 1, "dblpid": "books/acm/kim95/AnnevelinkACFHK95", "title": "Object SQL - A Language for the Design and Implementation of Object Databases.", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent", "misc": "2002-01-03 42-68 1995 Modern Database Systems db/books/collections/kim95.html#AnnevelinkACFHK95" }, "csx": { "id": 1, "csxid": "oai CiteSeerXPSU 10.1.1.39.1830", "title": "Object SQL - A Language for the Design and Implementation of Object Databases", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Dan Fishman Mike Heytens William Kent", "misc": "2009-04-13 ly, a function application expression consists of two expressions a function reference (labelled func_ref in Figure 3 line 2), and an argument (labelled arg). The func_ref expression evaluates to a (generic or specific) function identifier, which may be the same as the function that the expression is a part of, thus allowing recursive function invocations. The expression labelled arg evaluates to an arbitrary object or aggregate object. The semantics of evaluating function applications was discussed in detail in section 2. For example, to set the name of a person, we evaluate the following expression FunAssign(function name.person) (p1,'John') In this example, the first expression is itself a function call, applying the function FunAssign to the function name.person (an example of a specific function reference). This returns the oid of the function that sets a person's name, which is subsequently applied to a tuple of two elements, the oid of the person and the new name (a string o... CiteSeerX ACM Press 2009-04-13 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.39.1830 http //www.tu-chemnitz.de/~igrdb/docs/OpenODB/osql.ps.gz en 10.1.1.31.2534 10.1.1.28.4658 10.1.1.44.5947 10.1.1.39.199 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "csx": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems topics covered include active database rule models and languages, rule execution semantics, and implementation issues. 1 Introduction Conventional database systems are passive they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
+{ "dblp": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "csx": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema. The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer. When a global query is submitted, it is decomposed into two types of queries [1] -- subqueries, operating on sharable data items from local databases, -- assembling queries, consisting of, CiteSeerX 2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "csx": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of different types) over different processing entities. Such applications are referred to as transactional workflows. In this chapter we discuss the specification of such workflows and the issues involved in their execution. 1 What is a Workflow? Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX 2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 93, "csxid": "oai CiteSeerXPSU 10.1.1.42.9253", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-11 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of office information systems it is costly and difficult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"office objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to office software. In order to fully exploit the approach to achieve integrated office systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt t CiteSeerX ACM Press and Addison-Wesley 2009-04-11 2007-11-22 1988 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.42.9253 ftp //ftp.iam.unibe.ch/pub/scg/Papers/integratedOfficeSystems.ps.gz en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
index f6c6049..194ac75 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
@@ -1,3 +1,4 @@
+{ "a": "Query Processing in Multidatabase Systems.", "b": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "jacc": 0.527027f }
{ "a": "Transaction Management in Multidatabase Systems.", "b": "Overview of Multidatabase Transaction Management", "jacc": 0.55932206f }
{ "a": "Transaction Management in Multidatabase Systems.", "b": "Overview of Multidatabase Transaction Management", "jacc": 0.55932206f }
{ "a": "Active Database Systems.", "b": "Active Database Systems", "jacc": 0.95454544f }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.1.adm
index b528fe7..7594b6d 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.1.adm
@@ -1,6 +1,7 @@
{ "a": "Transaction Management in Multidatabase Systems.", "b": "Overview of Multidatabase Transaction Management" }
{ "a": "Transaction Management in Multidatabase Systems.", "b": "Overview of Multidatabase Transaction Management" }
{ "a": "Active Database Systems.", "b": "Active Database Systems" }
+{ "a": "Query Processing in Multidatabase Systems.", "b": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1." }
{ "a": "Specification and Execution of Transactional Workflows.", "b": "Specification and Execution of Transactional Workflows" }
{ "a": "Integrated Office Systems.", "b": "Integrated Office Systems" }
{ "a": "Integrated Office Systems.", "b": "Integrated Office Systems" }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.1.adm
index 29e66a2..2a29873 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.1.adm
@@ -1,3 +1,4 @@
+{ "a": "Query Processing in Multidatabase Systems.", "b": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "jacc": 0.5f }
{ "a": "Active Database Systems.", "b": "Active Database Systems", "jacc": 1.0f }
{ "a": "Specification and Execution of Transactional Workflows.", "b": "Specification and Execution of Transactional Workflows", "jacc": 1.0f }
{ "a": "Integrated Office Systems.", "b": "Integrated Office Systems", "jacc": 1.0f }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.1.adm
index 2bd52e3..d01a56f 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.1.adm
@@ -1,4 +1,5 @@
{ "a": "Active Database Systems.", "b": "Active Database Systems" }
+{ "a": "Query Processing in Multidatabase Systems.", "b": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1." }
{ "a": "Specification and Execution of Transactional Workflows.", "b": "Specification and Execution of Transactional Workflows" }
{ "a": "Integrated Office Systems.", "b": "Integrated Office Systems" }
{ "a": "Integrated Office Systems.", "b": "Integrated Office Systems" }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
index 568272c..6171f93 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
@@ -1,3 +1,4 @@
+{ "arec": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "brec": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema. The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer. When a global query is submitted, it is decomposed into two types of queries [1] -- subqueries, operating on sharable data items from local databases, -- assembling queries, consisting of, CiteSeerX 2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 0.527027f }
{ "arec": { "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }, "brec": { "id": 85, "csxid": "oai CiteSeerXPSU 10.1.1.37.8818", "title": "Overview of Multidatabase Transaction Management", "authors": "Yuri Breitbart Hector Garcia-Molina Avi Silberschatz", "misc": "2009-06-22 A multidatabase system (MDBS) is a facility that allows users access to data located in multiple autonomous database management systems (DBMSs). In such a system, global transactions are executed under the control of the MDBS. Independently, local transactions are executed under the control of the local DBMSs. Each local DBMS integrated by the MDBS may employ a different transaction management scheme. In addition, each local DBMS has complete control over all transactions (global and local) executing at its site, including the ability to abort at any point any of the transactions executing at its site. Typically, no design or internal DBMS structure changes are allowed in order to accommodate the MDBS. Furthermore, the local DBMSs may not be aware of each other, and, as a consequence, cannot coordinate their actions. Thus, traditional techniques for ensuring transaction atomicity and consistency in homogeneous distributed database systems may not be appropriate for an MDBS environment.... CiteSeerX 2009-06-22 2007-11-22 1992 text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.37.8818 ftp //ftp.cs.utexas.edu/pub/avi/UT-CS-TR-92-21.PS.Z en 10.1.1.101.8988 10.1.1.130.1772 10.1.1.38.6210 10.1.1.34.3768 10.1.1.36.1275 10.1.1.104.3430 10.1.1.112.244 10.1.1.94.9106 10.1.1.41.4043 10.1.1.49.5143 10.1.1.59.2034 10.1.1.53.875 10.1.1.137.5642 10.1.1.41.8832 10.1.1.21.1100 10.1.1.105.3626 10.1.1.44.773 10.1.1.21.2576 10.1.1.40.6484 10.1.1.144.2713 10.1.1.48.6718 10.1.1.16.6166 10.1.1.40.832 10.1.1.36.2660 10.1.1.30.3087 10.1.1.47.322 10.1.1.17.6532 10.1.1.33.2301 10.1.1.20.4306 10.1.1.47.6258 10.1.1.39.9212 10.1.1.46.4334 10.1.1.71.485 10.1.1.43.1405 10.1.1.49.1308 10.1.1.35.6530 10.1.1.42.5177 10.1.1.54.4068 10.1.1.133.3692 10.1.1.40.4220 10.1.1.48.7743 10.1.1.26.575 10.1.1.107.596 10.1.1.116.3495 10.1.1.33.2074 10.1.1.38.7229 10.1.1.59.4464 10.1.1.103.9562 10.1.1.36.5887 10.1.1.40.9658 10.1.1.53.6783 10.1.1.29.5010 10.1.1.107.876 10.1.1.46.2273 10.1.1.46.3657 10.1.1.49.5281 10.1.1.50.4114 10.1.1.63.3234 10.1.1.79.9607 10.1.1.83.4819 10.1.1.83.4980 10.1.1.84.8136 10.1.1.90.953 10.1.1.90.9785 10.1.1.92.2397 10.1.1.93.8911 10.1.1.94.3702 10.1.1.97.672 10.1.1.98.4604 10.1.1.117.6190 10.1.1.118.4814 10.1.1.130.880 10.1.1.137.1167 10.1.1.51.5111 10.1.1.45.2774 10.1.1.45.9165 10.1.1.40.4684 10.1.1.35.5866 10.1.1.38.3606 10.1.1.29.9166 10.1.1.31.3667 10.1.1.21.7181 10.1.1.33.2343 10.1.1.23.3117 10.1.1.24.7879 10.1.1.18.8936 10.1.1.19.3770 10.1.1.19.5246 10.1.1.12.3293 10.1.1.2.2325 10.1.1.60.116 10.1.1.140.5244 10.1.1.143.3448 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 0.55932206f }
{ "arec": { "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }, "brec": { "id": 86, "csxid": "oai CiteSeerXPSU 10.1.1.54.6302", "title": "Overview of Multidatabase Transaction Management", "authors": "Yuri Breitbart Hector Garcia-molina Avi Silberschatz", "misc": "2009-04-12 A multidatabase system (MDBS) is a facility that allows users access to data located in multiple autonomous database management systems (DBMSs). In such a system, global transactions are executed under the control of the MDBS. Independently, local transactions are executed under the control of the local DBMSs. Each local DBMS integrated by the MDBS may employ a different transaction management scheme. In addition, each local DBMS has complete control over all transactions (global and local) executing at its site, including the ability to abort at any point any of the transactions executing at its site. Typically, no design or internal DBMS structure changes are allowed in order to accommodate the MDBS. Furthermore, the local DBMSs may not be aware of each other, and, as a consequence, cannot coordinate their actions. Thus, traditional techniques for ensuring transaction atomicity and consistency in homogeneous distributed database systems may not be appropriate for an MDBS environment.... CiteSeerX 2009-04-12 2007-11-22 1992 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.54.6302 http //www-db.stanford.edu/pub/papers/multidatabase.ps en 10.1.1.101.8988 10.1.1.130.1772 10.1.1.38.6210 10.1.1.34.3768 10.1.1.36.1275 10.1.1.104.3430 10.1.1.112.244 10.1.1.94.9106 10.1.1.41.4043 10.1.1.49.5143 10.1.1.59.2034 10.1.1.53.875 10.1.1.137.5642 10.1.1.41.8832 10.1.1.21.1100 10.1.1.105.3626 10.1.1.44.773 10.1.1.21.2576 10.1.1.40.6484 10.1.1.144.2713 10.1.1.48.6718 10.1.1.16.6166 10.1.1.40.832 10.1.1.36.2660 10.1.1.30.3087 10.1.1.47.322 10.1.1.17.6532 10.1.1.33.2301 10.1.1.20.4306 10.1.1.47.6258 10.1.1.39.9212 10.1.1.46.4334 10.1.1.71.485 10.1.1.43.1405 10.1.1.49.1308 10.1.1.35.6530 10.1.1.42.5177 10.1.1.54.4068 10.1.1.133.3692 10.1.1.40.4220 10.1.1.48.7743 10.1.1.26.575 10.1.1.107.596 10.1.1.116.3495 10.1.1.33.2074 10.1.1.38.7229 10.1.1.59.4464 10.1.1.103.9562 10.1.1.36.5887 10.1.1.40.9658 10.1.1.53.6783 10.1.1.29.5010 10.1.1.107.876 10.1.1.46.2273 10.1.1.46.3657 10.1.1.49.5281 10.1.1.50.4114 10.1.1.63.3234 10.1.1.79.9607 10.1.1.83.4819 10.1.1.83.4980 10.1.1.84.8136 10.1.1.90.953 10.1.1.90.9785 10.1.1.92.2397 10.1.1.93.8911 10.1.1.94.3702 10.1.1.97.672 10.1.1.98.4604 10.1.1.117.6190 10.1.1.118.4814 10.1.1.130.880 10.1.1.137.1167 10.1.1.51.5111 10.1.1.45.2774 10.1.1.45.9165 10.1.1.40.4684 10.1.1.35.5866 10.1.1.38.3606 10.1.1.29.9166 10.1.1.31.3667 10.1.1.21.7181 10.1.1.33.2343 10.1.1.23.3117 10.1.1.24.7879 10.1.1.18.8936 10.1.1.19.3770 10.1.1.19.5246 10.1.1.12.3293 10.1.1.2.2325 10.1.1.60.116 10.1.1.140.5244 10.1.1.143.3448 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 0.55932206f }
{ "arec": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "brec": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems topics covered include active database rule models and languages, rule execution semantics, and implementation issues. 1 Introduction Conventional database systems are passive they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 0.95454544f }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard/ngram-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard/ngram-jaccard.1.adm
index f48c6c9..bb0d055 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard/ngram-jaccard.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard/ngram-jaccard.1.adm
@@ -1,6 +1,7 @@
{ "arec": { "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }, "brec": { "id": 85, "csxid": "oai CiteSeerXPSU 10.1.1.37.8818", "title": "Overview of Multidatabase Transaction Management", "authors": "Yuri Breitbart Hector Garcia-Molina Avi Silberschatz", "misc": "2009-06-22 A multidatabase system (MDBS) is a facility that allows users access to data located in multiple autonomous database management systems (DBMSs). In such a system, global transactions are executed under the control of the MDBS. Independently, local transactions are executed under the control of the local DBMSs. Each local DBMS integrated by the MDBS may employ a different transaction management scheme. In addition, each local DBMS has complete control over all transactions (global and local) executing at its site, including the ability to abort at any point any of the transactions executing at its site. Typically, no design or internal DBMS structure changes are allowed in order to accommodate the MDBS. Furthermore, the local DBMSs may not be aware of each other, and, as a consequence, cannot coordinate their actions. Thus, traditional techniques for ensuring transaction atomicity and consistency in homogeneous distributed database systems may not be appropriate for an MDBS environment.... CiteSeerX 2009-06-22 2007-11-22 1992 text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.37.8818 ftp //ftp.cs.utexas.edu/pub/avi/UT-CS-TR-92-21.PS.Z en 10.1.1.101.8988 10.1.1.130.1772 10.1.1.38.6210 10.1.1.34.3768 10.1.1.36.1275 10.1.1.104.3430 10.1.1.112.244 10.1.1.94.9106 10.1.1.41.4043 10.1.1.49.5143 10.1.1.59.2034 10.1.1.53.875 10.1.1.137.5642 10.1.1.41.8832 10.1.1.21.1100 10.1.1.105.3626 10.1.1.44.773 10.1.1.21.2576 10.1.1.40.6484 10.1.1.144.2713 10.1.1.48.6718 10.1.1.16.6166 10.1.1.40.832 10.1.1.36.2660 10.1.1.30.3087 10.1.1.47.322 10.1.1.17.6532 10.1.1.33.2301 10.1.1.20.4306 10.1.1.47.6258 10.1.1.39.9212 10.1.1.46.4334 10.1.1.71.485 10.1.1.43.1405 10.1.1.49.1308 10.1.1.35.6530 10.1.1.42.5177 10.1.1.54.4068 10.1.1.133.3692 10.1.1.40.4220 10.1.1.48.7743 10.1.1.26.575 10.1.1.107.596 10.1.1.116.3495 10.1.1.33.2074 10.1.1.38.7229 10.1.1.59.4464 10.1.1.103.9562 10.1.1.36.5887 10.1.1.40.9658 10.1.1.53.6783 10.1.1.29.5010 10.1.1.107.876 10.1.1.46.2273 10.1.1.46.3657 10.1.1.49.5281 10.1.1.50.4114 10.1.1.63.3234 10.1.1.79.9607 10.1.1.83.4819 10.1.1.83.4980 10.1.1.84.8136 10.1.1.90.953 10.1.1.90.9785 10.1.1.92.2397 10.1.1.93.8911 10.1.1.94.3702 10.1.1.97.672 10.1.1.98.4604 10.1.1.117.6190 10.1.1.118.4814 10.1.1.130.880 10.1.1.137.1167 10.1.1.51.5111 10.1.1.45.2774 10.1.1.45.9165 10.1.1.40.4684 10.1.1.35.5866 10.1.1.38.3606 10.1.1.29.9166 10.1.1.31.3667 10.1.1.21.7181 10.1.1.33.2343 10.1.1.23.3117 10.1.1.24.7879 10.1.1.18.8936 10.1.1.19.3770 10.1.1.19.5246 10.1.1.12.3293 10.1.1.2.2325 10.1.1.60.116 10.1.1.140.5244 10.1.1.143.3448 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "arec": { "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }, "brec": { "id": 86, "csxid": "oai CiteSeerXPSU 10.1.1.54.6302", "title": "Overview of Multidatabase Transaction Management", "authors": "Yuri Breitbart Hector Garcia-molina Avi Silberschatz", "misc": "2009-04-12 A multidatabase system (MDBS) is a facility that allows users access to data located in multiple autonomous database management systems (DBMSs). In such a system, global transactions are executed under the control of the MDBS. Independently, local transactions are executed under the control of the local DBMSs. Each local DBMS integrated by the MDBS may employ a different transaction management scheme. In addition, each local DBMS has complete control over all transactions (global and local) executing at its site, including the ability to abort at any point any of the transactions executing at its site. Typically, no design or internal DBMS structure changes are allowed in order to accommodate the MDBS. Furthermore, the local DBMSs may not be aware of each other, and, as a consequence, cannot coordinate their actions. Thus, traditional techniques for ensuring transaction atomicity and consistency in homogeneous distributed database systems may not be appropriate for an MDBS environment.... CiteSeerX 2009-04-12 2007-11-22 1992 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.54.6302 http //www-db.stanford.edu/pub/papers/multidatabase.ps en 10.1.1.101.8988 10.1.1.130.1772 10.1.1.38.6210 10.1.1.34.3768 10.1.1.36.1275 10.1.1.104.3430 10.1.1.112.244 10.1.1.94.9106 10.1.1.41.4043 10.1.1.49.5143 10.1.1.59.2034 10.1.1.53.875 10.1.1.137.5642 10.1.1.41.8832 10.1.1.21.1100 10.1.1.105.3626 10.1.1.44.773 10.1.1.21.2576 10.1.1.40.6484 10.1.1.144.2713 10.1.1.48.6718 10.1.1.16.6166 10.1.1.40.832 10.1.1.36.2660 10.1.1.30.3087 10.1.1.47.322 10.1.1.17.6532 10.1.1.33.2301 10.1.1.20.4306 10.1.1.47.6258 10.1.1.39.9212 10.1.1.46.4334 10.1.1.71.485 10.1.1.43.1405 10.1.1.49.1308 10.1.1.35.6530 10.1.1.42.5177 10.1.1.54.4068 10.1.1.133.3692 10.1.1.40.4220 10.1.1.48.7743 10.1.1.26.575 10.1.1.107.596 10.1.1.116.3495 10.1.1.33.2074 10.1.1.38.7229 10.1.1.59.4464 10.1.1.103.9562 10.1.1.36.5887 10.1.1.40.9658 10.1.1.53.6783 10.1.1.29.5010 10.1.1.107.876 10.1.1.46.2273 10.1.1.46.3657 10.1.1.49.5281 10.1.1.50.4114 10.1.1.63.3234 10.1.1.79.9607 10.1.1.83.4819 10.1.1.83.4980 10.1.1.84.8136 10.1.1.90.953 10.1.1.90.9785 10.1.1.92.2397 10.1.1.93.8911 10.1.1.94.3702 10.1.1.97.672 10.1.1.98.4604 10.1.1.117.6190 10.1.1.118.4814 10.1.1.130.880 10.1.1.137.1167 10.1.1.51.5111 10.1.1.45.2774 10.1.1.45.9165 10.1.1.40.4684 10.1.1.35.5866 10.1.1.38.3606 10.1.1.29.9166 10.1.1.31.3667 10.1.1.21.7181 10.1.1.33.2343 10.1.1.23.3117 10.1.1.24.7879 10.1.1.18.8936 10.1.1.19.3770 10.1.1.19.5246 10.1.1.12.3293 10.1.1.2.2325 10.1.1.60.116 10.1.1.140.5244 10.1.1.143.3448 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "arec": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "brec": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems topics covered include active database rule models and languages, rule execution semantics, and implementation issues. 1 Introduction Conventional database systems are passive they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
+{ "arec": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "brec": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema. The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer. When a global query is submitted, it is decomposed into two types of queries [1] -- subqueries, operating on sharable data items from local databases, -- assembling queries, consisting of, CiteSeerX 2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "arec": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "brec": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of different types) over different processing entities. Such applications are referred to as transactional workflows. In this chapter we discuss the specification of such workflows and the issues involved in their execution. 1 What is a Workflow? Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "arec": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "brec": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX 2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "arec": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "brec": { "id": 93, "csxid": "oai CiteSeerXPSU 10.1.1.42.9253", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-11 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of office information systems it is costly and difficult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"office objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to office software. In order to fully exploit the approach to achieve integrated office systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt t CiteSeerX ACM Press and Addison-Wesley 2009-04-11 2007-11-22 1988 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.42.9253 ftp //ftp.iam.unibe.ch/pub/scg/Papers/integratedOfficeSystems.ps.gz en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard-inline/word-jaccard-inline.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard-inline/word-jaccard-inline.1.adm
index 7e80ba2..2f6eb63 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard-inline/word-jaccard-inline.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard-inline/word-jaccard-inline.1.adm
@@ -1,3 +1,4 @@
+{ "arec": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "brec": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema. The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer. When a global query is submitted, it is decomposed into two types of queries [1] -- subqueries, operating on sharable data items from local databases, -- assembling queries, consisting of, CiteSeerX 2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 0.5f }
{ "arec": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "brec": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems topics covered include active database rule models and languages, rule execution semantics, and implementation issues. 1 Introduction Conventional database systems are passive they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 1.0f }
{ "arec": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "brec": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of different types) over different processing entities. Such applications are referred to as transactional workflows. In this chapter we discuss the specification of such workflows and the issues involved in their execution. 1 What is a Workflow? Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 1.0f }
{ "arec": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "brec": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX 2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 1.0f }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard/word-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard/word-jaccard.1.adm
index 3550136..3ca1911 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard/word-jaccard.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard/word-jaccard.1.adm
@@ -1,4 +1,5 @@
{ "arec": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "brec": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems topics covered include active database rule models and languages, rule execution semantics, and implementation issues. 1 Introduction Conventional database systems are passive they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
+{ "arec": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "brec": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema. The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer. When a global query is submitted, it is decomposed into two types of queries [1] -- subqueries, operating on sharable data items from local databases, -- assembling queries, consisting of, CiteSeerX 2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "arec": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "brec": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of different types) over different processing entities. Such applications are referred to as transactional workflows. In this chapter we discuss the specification of such workflows and the issues involved in their execution. 1 What is a Workflow? Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "arec": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "brec": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX 2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
{ "arec": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "brec": { "id": 93, "csxid": "oai CiteSeerXPSU 10.1.1.42.9253", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-11 Introduction New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of office information systems it is costly and difficult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"office objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to office software. In order to fully exploit the approach to achieve integrated office systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation object-oriented languages and systems exploit encapsulation in various ways in an attempt t CiteSeerX ACM Press and Addison-Wesley 2009-04-11 2007-11-22 1988 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.42.9253 ftp //ftp.iam.unibe.ch/pub/scg/Papers/integratedOfficeSystems.ps.gz en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
diff --git a/asterix-app/src/test/resources/runtimets/results/load/csv_01/csv_01.1.adm b/asterix-app/src/test/resources/runtimets/results/load/csv_01/csv_01.1.adm
new file mode 100644
index 0000000..b8d4151
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/load/csv_01/csv_01.1.adm
@@ -0,0 +1,8 @@
+{ "id": 1, "float": 0.89968276f, "double": 5.6256d, "date-before": "2013-08-07", "date-after": date("2013-08-07"), "time-before": "07:22:35", "time-after": time("07:22:35.000Z"), "datetime-before": "1979-02-25T23:48:27.034", "datetime-after": datetime("1979-02-25T23:48:27.034Z") }
+{ "id": 2, "float": 0.6690524f, "double": null, "date-before": "-1923-03-29", "date-after": date("-1923-03-29"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.002", "datetime-after": datetime("-1979-02-25T23:48:27.002Z") }
+{ "id": 3, "float": 0.57273304f, "double": 192674.0d, "date-before": "-1923-03-28", "date-after": date("-1923-03-28"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
+{ "id": 4, "float": null, "double": 192674.0d, "date-before": "-1923-03-27", "date-after": date("-1923-03-27"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
+{ "id": 5, "float": 0.57273304f, "double": 192674.0d, "date-before": null, "date-after": null, "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
+{ "id": 6, "float": 0.57273304f, "double": 192674.0d, "date-before": "-1923-03-25", "date-after": date("-1923-03-25"), "time-before": null, "time-after": null, "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
+{ "id": 7, "float": 0.57273304f, "double": 192674.0d, "date-before": "-1923-03-24", "date-after": date("-1923-03-24"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": null, "datetime-after": null }
+{ "id": 8, "float": null, "double": null, "date-before": null, "date-after": null, "time-before": null, "time-after": null, "datetime-before": null, "datetime-after": null }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/load/csv_02/csv_02.1.adm b/asterix-app/src/test/resources/runtimets/results/load/csv_02/csv_02.1.adm
new file mode 100644
index 0000000..44240dd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/load/csv_02/csv_02.1.adm
@@ -0,0 +1,8 @@
+{ "id": 1, "float": 0.89968276f, "double": 5.6256d, "date-string": "2013-08-07", "time-string": "07:22:35", "datetime-string": "1979-02-25T23:48:27.034" }
+{ "id": 2, "float": 0.6690524f, "double": null, "date-string": "-1923-03-29", "time-string": "19:33:34", "datetime-string": "-1979-02-25T23:48:27.002" }
+{ "id": 3, "float": 0.57273304f, "double": 192674.0d, "date-string": "-1923-03-28", "time-string": "19:33:34", "datetime-string": "-1979-02-25T23:48:27.001" }
+{ "id": 4, "float": null, "double": 192674.0d, "date-string": "-1923-03-27", "time-string": "19:33:34", "datetime-string": "-1979-02-25T23:48:27.001" }
+{ "id": 5, "float": 0.57273304f, "double": 192674.0d, "date-string": "", "time-string": "19:33:34", "datetime-string": "-1979-02-25T23:48:27.001" }
+{ "id": 6, "float": 0.57273304f, "double": 192674.0d, "date-string": "-1923-03-25", "time-string": "", "datetime-string": "-1979-02-25T23:48:27.001" }
+{ "id": 7, "float": 0.57273304f, "double": 192674.0d, "date-string": "-1923-03-24", "time-string": "19:33:34", "datetime-string": "" }
+{ "id": 8, "float": null, "double": null, "date-string": "", "time-string": "", "datetime-string": "" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/unary-minus_double_01/unary-minus_double_01.1.adm b/asterix-app/src/test/resources/runtimets/results/numeric/unary-minus_double_01/unary-minus_double_01.1.adm
deleted file mode 100644
index 3a72bb8..0000000
--- a/asterix-app/src/test/resources/runtimets/results/numeric/unary-minus_double_01/unary-minus_double_01.1.adm
+++ /dev/null
@@ -1 +0,0 @@
-{ "double1": 2.056E-29d, "double2": NaNd, "double3": -Infinityd, "double4": Infinityd }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/unary-minus_float_01/unary-minus_float_01.1.adm b/asterix-app/src/test/resources/runtimets/results/numeric/unary-minus_float_01/unary-minus_float_01.1.adm
deleted file mode 100644
index e6541ae..0000000
--- a/asterix-app/src/test/resources/runtimets/results/numeric/unary-minus_float_01/unary-minus_float_01.1.adm
+++ /dev/null
@@ -1 +0,0 @@
-{ "float1": 80.2f, "float2": NaNf, "float3": -Infinityf, "float4": Infinityf }
diff --git a/asterix-app/src/test/resources/runtimets/results/numeric/unary-minus_int_01/unary-minus_int_01.1.adm b/asterix-app/src/test/resources/runtimets/results/numeric/unary-minus_int_01/unary-minus_int_01.1.adm
deleted file mode 100644
index 4ecc59b..0000000
--- a/asterix-app/src/test/resources/runtimets/results/numeric/unary-minus_int_01/unary-minus_int_01.1.adm
+++ /dev/null
@@ -1 +0,0 @@
-{ "int8": -80i8, "int16": -160i16, "int32": -320, "int64": 640i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.adm
new file mode 100644
index 0000000..f14f6a7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.adm
@@ -0,0 +1,3 @@
+[ true, 0.5f ]
+[ true, 0.5f ]
+[ false, 0.0f ]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/day_of_week_01/day_of_week_01.1.adm b/asterix-app/src/test/resources/runtimets/results/temporal/day_of_week_01/day_of_week_01.1.adm
new file mode 100644
index 0000000..dff4aa2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/day_of_week_01/day_of_week_01.1.adm
@@ -0,0 +1 @@
+{ "1970-01-01": 4, "2013-08-06": 2, "-2013-08-06": 4, "1913-08-06T15:53:28Z": 3, "-1913-08-10T15:53:28Z": 7, "null": null }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin/interval_bin.1.adm b/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin/interval_bin.1.adm
new file mode 100644
index 0000000..3ba93a5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin/interval_bin.1.adm
@@ -0,0 +1 @@
+{ "bin1": interval-date("2010-01-01, 2011-01-01"), "bin2": interval-date("2010-01-01, 2011-01-01"), "bin3": interval-datetime("-1987-07-01T00:00:00.000Z, -1986-01-01T00:00:00.000Z"), "bin4": interval-datetime("-1987-11-19T12:00:00.000Z, -1987-11-20T00:00:00.000Z"), "bin5": interval-time("04:00:00.000Z, 06:00:00.000Z"), "bin6": null, "bin7": null, "bin8": null }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_0/interval_bin_gby_0.1.adm b/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_0/interval_bin_gby_0.1.adm
new file mode 100644
index 0000000..c21626b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_0/interval_bin_gby_0.1.adm
@@ -0,0 +1,4 @@
+{ "tbin": interval-datetime("-1990-01-01T00:00:00.000Z, -1970-01-01T00:00:00.000Z"), "count": 4i64 }
+{ "tbin": interval-datetime("-0990-01-01T00:00:00.000Z, -0970-01-01T00:00:00.000Z"), "count": 1i64 }
+{ "tbin": interval-datetime("1970-01-01T00:00:00.000Z, 1990-01-01T00:00:00.000Z"), "count": 5i64 }
+{ "tbin": interval-datetime("2010-01-01T00:00:00.000Z, 2030-01-01T00:00:00.000Z"), "count": 2i64 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_1/interval_bin_gby_1.1.adm b/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_1/interval_bin_gby_1.1.adm
new file mode 100644
index 0000000..b67989d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_1/interval_bin_gby_1.1.adm
@@ -0,0 +1,8 @@
+{ "tbin": interval-time("00:20:00.000Z, 00:30:00.000Z"), "count": 1i64 }
+{ "tbin": interval-time("09:30:00.000Z, 09:40:00.000Z"), "count": 1i64 }
+{ "tbin": interval-time("17:20:00.000Z, 17:30:00.000Z"), "count": 1i64 }
+{ "tbin": interval-time("18:00:00.000Z, 18:10:00.000Z"), "count": 1i64 }
+{ "tbin": interval-time("23:20:00.000Z, 23:30:00.000Z"), "count": 1i64 }
+{ "tbin": interval-time("23:30:00.000Z, 23:40:00.000Z"), "count": 1i64 }
+{ "tbin": interval-time("23:40:00.000Z, 23:50:00.000Z"), "count": 5i64 }
+{ "tbin": interval-time("23:50:00.000Z, 00:00:00.000Z"), "count": 1i64 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/parse_01/parse_01.1.adm b/asterix-app/src/test/resources/runtimets/results/temporal/parse_01/parse_01.1.adm
new file mode 100644
index 0000000..dee2631
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/parse_01/parse_01.1.adm
@@ -0,0 +1 @@
+{ "date1": date("2013-08-23"), "date2": date("-0012-08-12"), "date3": date("-1234-01-01"), "date4": date("-1980-11-09"), "date5": date("-1990-11-09"), "date6": date("2013-08-19"), "data7": date("2013-08-19"), "time1": time("08:23:49.000Z"), "time2": time("08:19:23.320Z"), "time3": time("20:19:23.320Z"), "time4": time("10:30:40.948Z"), "time5": time("10:30:40.948Z"), "datetime1": datetime("-1203-12-30T15:48:27.000Z"), "datetime2": datetime("-1203-12-30T23:48:27.392Z"), "datetime3": datetime("1723-12-03T23:59:23.392Z"), "datetime4": datetime("1723-12-04T03:59:23.392Z"), "datetime5": datetime("1723-12-04T03:59:23.392Z"), "datetime6": datetime("1970-01-02T03:59:23.392Z"), "datetime7": datetime("1723-12-04T03:59:23.392Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/parse_02/parse_02.1.adm b/asterix-app/src/test/resources/runtimets/results/temporal/parse_02/parse_02.1.adm
new file mode 100644
index 0000000..963e8d0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/parse_02/parse_02.1.adm
@@ -0,0 +1 @@
+{ "date-string-1": "-123/1/30", "date-string-2": "JAN 30, -0123", "date-string-3": "-0123/01/30", "time-string-1": "8.7.29.03 AM Z", "time-string-2": "08.07.29.030 AM Z", "datetime-string-1": "DEC 31 3:59:59.999 PM 137 Z", "datetime-string-2": "0137/DEC/31 3:59:59.999Z PM", "datetime-string-3": "0137-12-31T15:59:59.999Z" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.1.adm
rename to asterix-app/src/test/resources/runtimets/results/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q2_minimum_cost_supplier/q2_minimum_cost_supplier.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.1.adm
similarity index 84%
rename from asterix-app/src/test/resources/runtimets/results/tpch/q2_minimum_cost_supplier/q2_minimum_cost_supplier.1.adm
rename to asterix-app/src/test/resources/runtimets/results/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.1.adm
index af60345..a7d5b93 100644
--- a/asterix-app/src/test/resources/runtimets/results/tpch/q2_minimum_cost_supplier/q2_minimum_cost_supplier.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.1.adm
@@ -1,6 +1,8 @@
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 2, "p_mfgr": "Manufacturer#1", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 4, "p_mfgr": "Manufacturer#3", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 22, "p_mfgr": "Manufacturer#4", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
+{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 35, "p_mfgr": "Manufacturer#4", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
+{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 38, "p_mfgr": "Manufacturer#4", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 62, "p_mfgr": "Manufacturer#3", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 79, "p_mfgr": "Manufacturer#4", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
{ "s_acctbal": 6820.35d, "s_name": "Supplier#000000007", "n_name": "UNITED KINGDOM", "p_partkey": 94, "p_mfgr": "Manufacturer#3", "s_address": "s,4TicNGB4uO6PaSqNBUq", "s_phone": "33-990-965-2201", "s_comment": "s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit" }
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q3_shipping_priority_nt/q3_shipping_priority_nt.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.1.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/tpch/q3_shipping_priority_nt/q3_shipping_priority_nt.1.adm
rename to asterix-app/src/test/resources/runtimets/results/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q4_order_priority/q4_order_priority.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q04_order_priority/q04_order_priority.1.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/tpch/q4_order_priority/q4_order_priority.1.adm
rename to asterix-app/src/test/resources/runtimets/results/tpch/q04_order_priority/q04_order_priority.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q5_local_supplier_volume/q5_local_supplier_volume.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q05_local_supplier_volume/q05_local_supplier_volume.1.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/tpch/q5_local_supplier_volume/q5_local_supplier_volume.1.adm
rename to asterix-app/src/test/resources/runtimets/results/tpch/q05_local_supplier_volume/q05_local_supplier_volume.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.1.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.1.adm
rename to asterix-app/src/test/resources/runtimets/results/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q7_volume_shipping/q7_volume_shipping.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q07_volume_shipping/q07_volume_shipping.1.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/tpch/q7_volume_shipping/q7_volume_shipping.1.adm
rename to asterix-app/src/test/resources/runtimets/results/tpch/q07_volume_shipping/q07_volume_shipping.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q8_national_market_share/q8_national_market_share.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q08_national_market_share/q08_national_market_share.1.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/results/tpch/q8_national_market_share/q8_national_market_share.1.adm
rename to asterix-app/src/test/resources/runtimets/results/tpch/q08_national_market_share/q08_national_market_share.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.1.adm
new file mode 100644
index 0000000..e9f3f47
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.1.adm
@@ -0,0 +1,59 @@
+{ "nation": "ARGENTINA", "o_year": 1997, "sum_profit": 18247.873399999993d }
+{ "nation": "ARGENTINA", "o_year": 1996, "sum_profit": 7731.089399999995d }
+{ "nation": "ARGENTINA", "o_year": 1995, "sum_profit": 134490.5697d }
+{ "nation": "ARGENTINA", "o_year": 1994, "sum_profit": 36767.101500000004d }
+{ "nation": "ARGENTINA", "o_year": 1993, "sum_profit": 35857.08d }
+{ "nation": "ARGENTINA", "o_year": 1992, "sum_profit": 35740.0d }
+{ "nation": "ETHIOPIA", "o_year": 1998, "sum_profit": 2758.7801999999992d }
+{ "nation": "ETHIOPIA", "o_year": 1997, "sum_profit": 19419.294599999994d }
+{ "nation": "ETHIOPIA", "o_year": 1995, "sum_profit": 51231.87439999999d }
+{ "nation": "ETHIOPIA", "o_year": 1994, "sum_profit": 3578.9478999999974d }
+{ "nation": "ETHIOPIA", "o_year": 1992, "sum_profit": 1525.8234999999986d }
+{ "nation": "IRAN", "o_year": 1998, "sum_profit": 37817.229600000006d }
+{ "nation": "IRAN", "o_year": 1997, "sum_profit": 52643.77359999999d }
+{ "nation": "IRAN", "o_year": 1996, "sum_profit": 70143.77609999999d }
+{ "nation": "IRAN", "o_year": 1995, "sum_profit": 84094.58260000001d }
+{ "nation": "IRAN", "o_year": 1994, "sum_profit": 18140.925599999995d }
+{ "nation": "IRAN", "o_year": 1993, "sum_profit": 78655.1676d }
+{ "nation": "IRAN", "o_year": 1992, "sum_profit": 87142.2396d }
+{ "nation": "IRAQ", "o_year": 1998, "sum_profit": 22860.8082d }
+{ "nation": "IRAQ", "o_year": 1997, "sum_profit": 93676.24359999999d }
+{ "nation": "IRAQ", "o_year": 1996, "sum_profit": 45103.3242d }
+{ "nation": "IRAQ", "o_year": 1994, "sum_profit": 36010.728599999995d }
+{ "nation": "IRAQ", "o_year": 1993, "sum_profit": 33221.9399d }
+{ "nation": "IRAQ", "o_year": 1992, "sum_profit": 47755.05900000001d }
+{ "nation": "KENYA", "o_year": 1998, "sum_profit": 44194.831999999995d }
+{ "nation": "KENYA", "o_year": 1997, "sum_profit": 57578.3626d }
+{ "nation": "KENYA", "o_year": 1996, "sum_profit": 59195.9021d }
+{ "nation": "KENYA", "o_year": 1995, "sum_profit": 79262.6278d }
+{ "nation": "KENYA", "o_year": 1994, "sum_profit": 102360.66609999999d }
+{ "nation": "KENYA", "o_year": 1993, "sum_profit": 128422.01959999999d }
+{ "nation": "KENYA", "o_year": 1992, "sum_profit": 181517.20890000003d }
+{ "nation": "MOROCCO", "o_year": 1998, "sum_profit": 41797.823199999984d }
+{ "nation": "MOROCCO", "o_year": 1997, "sum_profit": 23685.801799999997d }
+{ "nation": "MOROCCO", "o_year": 1996, "sum_profit": 62115.19579999999d }
+{ "nation": "MOROCCO", "o_year": 1995, "sum_profit": 42442.64300000001d }
+{ "nation": "MOROCCO", "o_year": 1994, "sum_profit": 48655.87800000001d }
+{ "nation": "MOROCCO", "o_year": 1993, "sum_profit": 22926.744400000003d }
+{ "nation": "MOROCCO", "o_year": 1992, "sum_profit": 32239.8088d }
+{ "nation": "PERU", "o_year": 1998, "sum_profit": 86999.36459999997d }
+{ "nation": "PERU", "o_year": 1997, "sum_profit": 121110.41070000001d }
+{ "nation": "PERU", "o_year": 1996, "sum_profit": 177040.40759999998d }
+{ "nation": "PERU", "o_year": 1995, "sum_profit": 122247.94519999999d }
+{ "nation": "PERU", "o_year": 1994, "sum_profit": 88046.2533d }
+{ "nation": "PERU", "o_year": 1993, "sum_profit": 49379.813799999996d }
+{ "nation": "PERU", "o_year": 1992, "sum_profit": 80646.86050000001d }
+{ "nation": "UNITED KINGDOM", "o_year": 1998, "sum_profit": 50577.25560000001d }
+{ "nation": "UNITED KINGDOM", "o_year": 1997, "sum_profit": 114288.86049999998d }
+{ "nation": "UNITED KINGDOM", "o_year": 1996, "sum_profit": 147684.46480000002d }
+{ "nation": "UNITED KINGDOM", "o_year": 1995, "sum_profit": 225267.6576d }
+{ "nation": "UNITED KINGDOM", "o_year": 1994, "sum_profit": 140595.58639999997d }
+{ "nation": "UNITED KINGDOM", "o_year": 1993, "sum_profit": 322548.49210000003d }
+{ "nation": "UNITED KINGDOM", "o_year": 1992, "sum_profit": 67747.88279999999d }
+{ "nation": "UNITED STATES", "o_year": 1998, "sum_profit": 3957.0431999999996d }
+{ "nation": "UNITED STATES", "o_year": 1997, "sum_profit": 94729.5704d }
+{ "nation": "UNITED STATES", "o_year": 1996, "sum_profit": 79297.8567d }
+{ "nation": "UNITED STATES", "o_year": 1995, "sum_profit": 62201.23360000001d }
+{ "nation": "UNITED STATES", "o_year": 1994, "sum_profit": 43075.62989999999d }
+{ "nation": "UNITED STATES", "o_year": 1993, "sum_profit": 27168.486199999996d }
+{ "nation": "UNITED STATES", "o_year": 1992, "sum_profit": 34092.366d }
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q11_important_stock/q11_important_stock.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q11_important_stock/q11_important_stock.1.adm
index 42b3861..0fc28fb 100644
--- a/asterix-app/src/test/resources/runtimets/results/tpch/q11_important_stock/q11_important_stock.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/tpch/q11_important_stock/q11_important_stock.1.adm
@@ -1,13 +1,13 @@
{ "partkey": 25, "part_value": 2.832302068E7d }
{ "partkey": 124, "part_value": 2.59627599E7d }
-{ "partkey": 175, "part_value": 2.3853953630000003E7d }
+{ "partkey": 175, "part_value": 2.385395363E7d }
{ "partkey": 197, "part_value": 2.248551967E7d }
{ "partkey": 163, "part_value": 2.099460571E7d }
{ "partkey": 160, "part_value": 2.00232846E7d }
-{ "partkey": 82, "part_value": 1.9919213349999998E7d }
+{ "partkey": 82, "part_value": 1.991921335E7d }
{ "partkey": 169, "part_value": 1.898734723E7d }
{ "partkey": 29, "part_value": 1.867279344E7d }
-{ "partkey": 26, "part_value": 1.8612458270000003E7d }
+{ "partkey": 26, "part_value": 1.861245827E7d }
{ "partkey": 73, "part_value": 1.827170729E7d }
{ "partkey": 161, "part_value": 1.7987463009999998E7d }
{ "partkey": 75, "part_value": 1.7959598009999998E7d }
@@ -16,28 +16,28 @@
{ "partkey": 69, "part_value": 1.728526943E7d }
{ "partkey": 111, "part_value": 1.7083882619999997E7d }
{ "partkey": 171, "part_value": 1.635442066E7d }
-{ "partkey": 166, "part_value": 1.6351893739999998E7d }
+{ "partkey": 166, "part_value": 1.6351893740000002E7d }
{ "partkey": 77, "part_value": 1.598059909E7d }
{ "partkey": 78, "part_value": 1.58768992E7d }
{ "partkey": 143, "part_value": 1.585686159E7d }
-{ "partkey": 17, "part_value": 1.5474261120000001E7d }
-{ "partkey": 109, "part_value": 1.505468262E7d }
+{ "partkey": 17, "part_value": 1.547426112E7d }
+{ "partkey": 109, "part_value": 1.5054682620000001E7d }
{ "partkey": 105, "part_value": 1.5053163809999999E7d }
{ "partkey": 96, "part_value": 1.495213259E7d }
{ "partkey": 146, "part_value": 1.481075944E7d }
{ "partkey": 136, "part_value": 1.465496775E7d }
{ "partkey": 116, "part_value": 1.4432091339999998E7d }
-{ "partkey": 128, "part_value": 1.439355526E7d }
-{ "partkey": 142, "part_value": 1.4220399040000001E7d }
+{ "partkey": 128, "part_value": 1.4393555259999998E7d }
+{ "partkey": 142, "part_value": 1.422039904E7d }
{ "partkey": 121, "part_value": 1.420032605E7d }
{ "partkey": 30, "part_value": 1.416313241E7d }
{ "partkey": 16, "part_value": 1.413646503E7d }
-{ "partkey": 198, "part_value": 1.4135353350000001E7d }
+{ "partkey": 198, "part_value": 1.413535335E7d }
{ "partkey": 79, "part_value": 1.38652287E7d }
{ "partkey": 90, "part_value": 1.373279748E7d }
{ "partkey": 32, "part_value": 1.369962979E7d }
{ "partkey": 74, "part_value": 1.338871111E7d }
-{ "partkey": 1, "part_value": 1.3378707239999998E7d }
+{ "partkey": 1, "part_value": 1.337870724E7d }
{ "partkey": 89, "part_value": 1.337148041E7d }
{ "partkey": 22, "part_value": 1.3354991740000002E7d }
{ "partkey": 186, "part_value": 1.317604077E7d }
@@ -45,13 +45,13 @@
{ "partkey": 14, "part_value": 1.299397721E7d }
{ "partkey": 93, "part_value": 1.299298218E7d }
{ "partkey": 168, "part_value": 1.299041501E7d }
-{ "partkey": 99, "part_value": 1.275004679E7d }
+{ "partkey": 99, "part_value": 1.2750046790000001E7d }
{ "partkey": 167, "part_value": 1.268255069E7d }
{ "partkey": 2, "part_value": 1.258471636E7d }
{ "partkey": 182, "part_value": 1.256239411E7d }
{ "partkey": 61, "part_value": 1.253677656E7d }
{ "partkey": 112, "part_value": 1.234957975E7d }
-{ "partkey": 178, "part_value": 1.226030174E7d }
+{ "partkey": 178, "part_value": 1.2260301739999998E7d }
{ "partkey": 172, "part_value": 1.219775193E7d }
{ "partkey": 165, "part_value": 1.219746506E7d }
{ "partkey": 184, "part_value": 1.216784393E7d }
@@ -59,14 +59,14 @@
{ "partkey": 153, "part_value": 1.2119354219999999E7d }
{ "partkey": 95, "part_value": 1.20468895E7d }
{ "partkey": 11, "part_value": 1.2007151559999999E7d }
-{ "partkey": 125, "part_value": 1.200347611E7d }
-{ "partkey": 154, "part_value": 1.1851133850000001E7d }
+{ "partkey": 125, "part_value": 1.2003476109999998E7d }
+{ "partkey": 154, "part_value": 1.185113385E7d }
{ "partkey": 15, "part_value": 1.1798438790000001E7d }
{ "partkey": 67, "part_value": 1.178579951E7d }
{ "partkey": 8, "part_value": 1.1707892620000001E7d }
{ "partkey": 87, "part_value": 1.168637671E7d }
-{ "partkey": 134, "part_value": 1.168358693E7d }
-{ "partkey": 130, "part_value": 1.168246149E7d }
+{ "partkey": 134, "part_value": 1.1683586929999998E7d }
+{ "partkey": 130, "part_value": 1.1682461489999998E7d }
{ "partkey": 43, "part_value": 1.161150462E7d }
{ "partkey": 102, "part_value": 1.151554211E7d }
{ "partkey": 21, "part_value": 1.141066856E7d }
@@ -76,7 +76,7 @@
{ "partkey": 173, "part_value": 1.102677486E7d }
{ "partkey": 94, "part_value": 1.092440116E7d }
{ "partkey": 3, "part_value": 1.075814545E7d }
-{ "partkey": 103, "part_value": 1.0691221600000001E7d }
+{ "partkey": 103, "part_value": 1.06912216E7d }
{ "partkey": 158, "part_value": 1.067861635E7d }
{ "partkey": 49, "part_value": 1.06445572E7d }
{ "partkey": 139, "part_value": 1.044045371E7d }
@@ -89,30 +89,30 @@
{ "partkey": 33, "part_value": 1.005296264E7d }
{ "partkey": 28, "part_value": 1.005234286E7d }
{ "partkey": 40, "part_value": 9927827.77d }
-{ "partkey": 199, "part_value": 9907803.56d }
+{ "partkey": 199, "part_value": 9907803.559999999d }
{ "partkey": 193, "part_value": 9869674.77d }
{ "partkey": 106, "part_value": 9869361.73d }
-{ "partkey": 108, "part_value": 9868370.31d }
+{ "partkey": 108, "part_value": 9868370.309999999d }
{ "partkey": 183, "part_value": 9855564.82d }
-{ "partkey": 70, "part_value": 9700431.940000001d }
-{ "partkey": 48, "part_value": 9655921.879999999d }
+{ "partkey": 70, "part_value": 9700431.94d }
+{ "partkey": 48, "part_value": 9655921.88d }
{ "partkey": 118, "part_value": 9622756.15d }
{ "partkey": 13, "part_value": 9592610.32d }
{ "partkey": 83, "part_value": 9543465.08d }
{ "partkey": 159, "part_value": 9519909.44d }
{ "partkey": 147, "part_value": 9513932.18d }
-{ "partkey": 45, "part_value": 9423874.469999999d }
+{ "partkey": 45, "part_value": 9423874.47d }
{ "partkey": 117, "part_value": 9408426.72d }
-{ "partkey": 135, "part_value": 9311247.28d }
+{ "partkey": 135, "part_value": 9311247.280000001d }
{ "partkey": 185, "part_value": 9305341.780000001d }
{ "partkey": 131, "part_value": 9223742.49d }
{ "partkey": 7, "part_value": 9175528.21d }
{ "partkey": 71, "part_value": 9167712.04d }
-{ "partkey": 100, "part_value": 9131099.53d }
+{ "partkey": 100, "part_value": 9131099.530000001d }
{ "partkey": 76, "part_value": 9092927.11d }
{ "partkey": 53, "part_value": 8979121.97d }
-{ "partkey": 141, "part_value": 8686511.12d }
-{ "partkey": 64, "part_value": 8627897.29d }
+{ "partkey": 141, "part_value": 8686511.120000001d }
+{ "partkey": 64, "part_value": 8627897.290000001d }
{ "partkey": 101, "part_value": 8521762.0d }
{ "partkey": 176, "part_value": 8510175.88d }
{ "partkey": 19, "part_value": 8481679.5d }
@@ -120,44 +120,43 @@
{ "partkey": 91, "part_value": 8460636.52d }
{ "partkey": 132, "part_value": 8416851.239999998d }
{ "partkey": 113, "part_value": 8405217.96d }
-{ "partkey": 51, "part_value": 8247118.5d }
+{ "partkey": 51, "part_value": 8247118.499999999d }
{ "partkey": 41, "part_value": 8187897.16d }
{ "partkey": 55, "part_value": 8092552.890000001d }
-{ "partkey": 72, "part_value": 8007155.300000001d }
+{ "partkey": 72, "part_value": 8007155.3d }
{ "partkey": 115, "part_value": 7954624.0d }
{ "partkey": 170, "part_value": 7895241.609999999d }
{ "partkey": 114, "part_value": 7832023.28d }
-{ "partkey": 37, "part_value": 7809598.66d }
-{ "partkey": 54, "part_value": 7578243.789999999d }
+{ "partkey": 37, "part_value": 7809598.659999999d }
+{ "partkey": 54, "part_value": 7578243.79d }
{ "partkey": 180, "part_value": 7531794.4799999995d }
-{ "partkey": 60, "part_value": 7508961.6899999995d }
-{ "partkey": 31, "part_value": 7433034.24d }
+{ "partkey": 60, "part_value": 7508961.69d }
+{ "partkey": 31, "part_value": 7433034.240000001d }
{ "partkey": 35, "part_value": 7132671.49d }
{ "partkey": 140, "part_value": 7122050.08d }
{ "partkey": 150, "part_value": 7106237.92d }
{ "partkey": 107, "part_value": 7082828.68d }
{ "partkey": 123, "part_value": 7049500.720000001d }
-{ "partkey": 190, "part_value": 7017966.899999999d }
+{ "partkey": 190, "part_value": 7017966.9d }
{ "partkey": 120, "part_value": 6920857.090000001d }
{ "partkey": 196, "part_value": 6905182.43d }
{ "partkey": 177, "part_value": 6887257.27d }
{ "partkey": 126, "part_value": 6813302.029999999d }
{ "partkey": 122, "part_value": 6812763.34d }
-{ "partkey": 200, "part_value": 6780024.529999999d }
+{ "partkey": 200, "part_value": 6780024.53d }
{ "partkey": 157, "part_value": 6766365.680000001d }
{ "partkey": 63, "part_value": 6724960.14d }
{ "partkey": 38, "part_value": 6667789.55d }
-{ "partkey": 58, "part_value": 6640619.38d }
-{ "partkey": 145, "part_value": 6633786.590000001d }
+{ "partkey": 58, "part_value": 6640619.380000001d }
+{ "partkey": 145, "part_value": 6633786.59d }
{ "partkey": 144, "part_value": 6546945.92d }
{ "partkey": 20, "part_value": 6533101.39d }
{ "partkey": 127, "part_value": 6483139.620000001d }
{ "partkey": 10, "part_value": 6433776.51d }
-{ "partkey": 36, "part_value": 6410209.249999999d }
-{ "partkey": 47, "part_value": 6407355.37d }
+{ "partkey": 47, "part_value": 6407355.369999999d }
{ "partkey": 191, "part_value": 6347187.43d }
{ "partkey": 137, "part_value": 6180452.85d }
-{ "partkey": 56, "part_value": 6145826.600000001d }
+{ "partkey": 56, "part_value": 6145826.6d }
{ "partkey": 104, "part_value": 6134341.85d }
{ "partkey": 44, "part_value": 6038126.66d }
{ "partkey": 97, "part_value": 6036047.1899999995d }
@@ -167,7 +166,7 @@
{ "partkey": 52, "part_value": 5680644.4799999995d }
{ "partkey": 155, "part_value": 5552007.57d }
{ "partkey": 92, "part_value": 5489588.279999999d }
-{ "partkey": 5, "part_value": 5461046.93d }
+{ "partkey": 5, "part_value": 5461046.930000001d }
{ "partkey": 18, "part_value": 5456316.21d }
{ "partkey": 149, "part_value": 5367514.63d }
{ "partkey": 110, "part_value": 5261352.11d }
@@ -175,26 +174,27 @@
{ "partkey": 6, "part_value": 5120392.470000001d }
{ "partkey": 148, "part_value": 5061589.27d }
{ "partkey": 42, "part_value": 4957032.47d }
-{ "partkey": 119, "part_value": 4954403.48d }
+{ "partkey": 119, "part_value": 4954403.4799999995d }
{ "partkey": 84, "part_value": 4891082.38d }
-{ "partkey": 65, "part_value": 4834763.090000001d }
+{ "partkey": 65, "part_value": 4834763.09d }
{ "partkey": 66, "part_value": 4719253.369999999d }
-{ "partkey": 179, "part_value": 4610607.92d }
+{ "partkey": 179, "part_value": 4610607.919999999d }
{ "partkey": 23, "part_value": 4531731.12d }
{ "partkey": 68, "part_value": 4504770.61d }
{ "partkey": 27, "part_value": 4371849.52d }
-{ "partkey": 129, "part_value": 3997604.7800000003d }
+{ "partkey": 36, "part_value": 4036576.8999999994d }
+{ "partkey": 129, "part_value": 3997604.78d }
{ "partkey": 195, "part_value": 3817436.31d }
{ "partkey": 59, "part_value": 3765210.2100000004d }
{ "partkey": 57, "part_value": 3739347.12d }
{ "partkey": 138, "part_value": 3567425.75d }
-{ "partkey": 174, "part_value": 3484708.31d }
+{ "partkey": 174, "part_value": 3484708.3100000005d }
{ "partkey": 164, "part_value": 3462215.0d }
{ "partkey": 81, "part_value": 3421610.42d }
{ "partkey": 46, "part_value": 3398443.33d }
{ "partkey": 85, "part_value": 3338711.3899999997d }
-{ "partkey": 50, "part_value": 3145791.97d }
+{ "partkey": 50, "part_value": 3145791.9699999997d }
{ "partkey": 88, "part_value": 3117730.2399999998d }
{ "partkey": 151, "part_value": 2727444.22d }
{ "partkey": 152, "part_value": 1837809.1700000002d }
-{ "partkey": 133, "part_value": 1517282.33d }
+{ "partkey": 133, "part_value": 1517282.3299999998d }
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.1.adm
index 4e06408..8247e9f 100644
--- a/asterix-app/src/test/resources/runtimets/results/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.1.adm
@@ -2,10 +2,12 @@
{ "p_brand": "Brand#11", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
@@ -16,19 +18,17 @@
{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#34", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#44", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
-{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 2i64 }
-{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 2i64 }
-{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 2i64 }
-{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 2i64 }
-{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 1i64 }
-{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 1i64 }
-{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 1i64 }
-{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 1i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 3i64 }
diff --git a/asterix-app/src/test/resources/runtimets/results/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.1.adm b/asterix-app/src/test/resources/runtimets/results/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.1.adm
deleted file mode 100644
index aec4b5e..0000000
--- a/asterix-app/src/test/resources/runtimets/results/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.1.adm
+++ /dev/null
@@ -1,59 +0,0 @@
-{ "nation": "ARGENTINA", "o_year": 1997, "sum_profit": 18247.873399999993d }
-{ "nation": "ARGENTINA", "o_year": 1996, "sum_profit": 7731.089399999995d }
-{ "nation": "ARGENTINA", "o_year": 1995, "sum_profit": 134490.5697d }
-{ "nation": "ARGENTINA", "o_year": 1994, "sum_profit": 36767.101500000004d }
-{ "nation": "ARGENTINA", "o_year": 1993, "sum_profit": 35857.08d }
-{ "nation": "ARGENTINA", "o_year": 1992, "sum_profit": 35740.0d }
-{ "nation": "ETHIOPIA", "o_year": 1998, "sum_profit": 2758.7801999999992d }
-{ "nation": "ETHIOPIA", "o_year": 1997, "sum_profit": 19419.294599999997d }
-{ "nation": "ETHIOPIA", "o_year": 1995, "sum_profit": 51231.87439999999d }
-{ "nation": "ETHIOPIA", "o_year": 1994, "sum_profit": 3578.9478999999974d }
-{ "nation": "ETHIOPIA", "o_year": 1992, "sum_profit": 1525.8234999999986d }
-{ "nation": "IRAN", "o_year": 1998, "sum_profit": 37817.229600000006d }
-{ "nation": "IRAN", "o_year": 1997, "sum_profit": 52643.77359999999d }
-{ "nation": "IRAN", "o_year": 1996, "sum_profit": 70143.77609999999d }
-{ "nation": "IRAN", "o_year": 1995, "sum_profit": 84094.58260000001d }
-{ "nation": "IRAN", "o_year": 1994, "sum_profit": 18140.925599999995d }
-{ "nation": "IRAN", "o_year": 1993, "sum_profit": 78655.1676d }
-{ "nation": "IRAN", "o_year": 1992, "sum_profit": 87142.2396d }
-{ "nation": "IRAQ", "o_year": 1998, "sum_profit": 22860.8082d }
-{ "nation": "IRAQ", "o_year": 1997, "sum_profit": 93676.24359999999d }
-{ "nation": "IRAQ", "o_year": 1996, "sum_profit": 45103.3242d }
-{ "nation": "IRAQ", "o_year": 1994, "sum_profit": 36010.728599999995d }
-{ "nation": "IRAQ", "o_year": 1993, "sum_profit": 33221.9399d }
-{ "nation": "IRAQ", "o_year": 1992, "sum_profit": 47755.05900000001d }
-{ "nation": "KENYA", "o_year": 1998, "sum_profit": 47441.812d }
-{ "nation": "KENYA", "o_year": 1997, "sum_profit": 118144.65099999998d }
-{ "nation": "KENYA", "o_year": 1996, "sum_profit": 75090.5437d }
-{ "nation": "KENYA", "o_year": 1995, "sum_profit": 87626.6791d }
-{ "nation": "KENYA", "o_year": 1994, "sum_profit": 130302.4071d }
-{ "nation": "KENYA", "o_year": 1993, "sum_profit": 185952.9548d }
-{ "nation": "KENYA", "o_year": 1992, "sum_profit": 310009.1845d }
-{ "nation": "MOROCCO", "o_year": 1998, "sum_profit": 172158.9448d }
-{ "nation": "MOROCCO", "o_year": 1997, "sum_profit": 92247.75379999999d }
-{ "nation": "MOROCCO", "o_year": 1996, "sum_profit": 273187.9262d }
-{ "nation": "MOROCCO", "o_year": 1995, "sum_profit": 380050.2938d }
-{ "nation": "MOROCCO", "o_year": 1994, "sum_profit": 262538.8032d }
-{ "nation": "MOROCCO", "o_year": 1993, "sum_profit": 211233.76599999992d }
-{ "nation": "MOROCCO", "o_year": 1992, "sum_profit": 336202.0852d }
-{ "nation": "PERU", "o_year": 1998, "sum_profit": 143951.22939999998d }
-{ "nation": "PERU", "o_year": 1997, "sum_profit": 197680.59720000002d }
-{ "nation": "PERU", "o_year": 1996, "sum_profit": 236156.92649999997d }
-{ "nation": "PERU", "o_year": 1995, "sum_profit": 190426.59480000002d }
-{ "nation": "PERU", "o_year": 1994, "sum_profit": 118171.16209999999d }
-{ "nation": "PERU", "o_year": 1993, "sum_profit": 102292.5426d }
-{ "nation": "PERU", "o_year": 1992, "sum_profit": 135197.25370000003d }
-{ "nation": "UNITED KINGDOM", "o_year": 1998, "sum_profit": 115488.22360000003d }
-{ "nation": "UNITED KINGDOM", "o_year": 1997, "sum_profit": 224769.43809999994d }
-{ "nation": "UNITED KINGDOM", "o_year": 1996, "sum_profit": 367799.3432d }
-{ "nation": "UNITED KINGDOM", "o_year": 1995, "sum_profit": 563229.4064d }
-{ "nation": "UNITED KINGDOM", "o_year": 1994, "sum_profit": 305695.22239999997d }
-{ "nation": "UNITED KINGDOM", "o_year": 1993, "sum_profit": 762202.0801000001d }
-{ "nation": "UNITED KINGDOM", "o_year": 1992, "sum_profit": 109066.47399999999d }
-{ "nation": "UNITED STATES", "o_year": 1998, "sum_profit": 3957.0431999999996d }
-{ "nation": "UNITED STATES", "o_year": 1997, "sum_profit": 94729.5704d }
-{ "nation": "UNITED STATES", "o_year": 1996, "sum_profit": 79297.8567d }
-{ "nation": "UNITED STATES", "o_year": 1995, "sum_profit": 62201.23360000001d }
-{ "nation": "UNITED STATES", "o_year": 1994, "sum_profit": 43075.6299d }
-{ "nation": "UNITED STATES", "o_year": 1993, "sum_profit": 27168.486199999992d }
-{ "nation": "UNITED STATES", "o_year": 1992, "sum_profit": 34092.366d }
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_int_01/unary-minus_int_01.2.update.aql b/asterix-app/src/test/resources/runtimets/results/types/record01/record01.1.adm
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/numeric/unary-minus_int_01/unary-minus_int_01.2.update.aql
rename to asterix-app/src/test/resources/runtimets/results/types/record01/record01.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 1fefaf0..707fb15 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -15,6 +15,11 @@
<test-suite xmlns="urn:xml.testframework.asterix.ics.uci.edu" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
<test-group name="aggregate">
<test-case FilePath="aggregate">
+ <compilation-unit name="issue531_string_min_max">
+ <output-dir compare="Text">issue531_string_min_max</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
<compilation-unit name="agg_null">
<output-dir compare="Text">agg_null</output-dir>
</compilation-unit>
@@ -965,8 +970,8 @@
</compilation-unit>
</test-case>
<test-case FilePath="dml">
- <compilation-unit name="insert-into-empty-dataset-with-index">
- <output-dir compare="Text">insert-into-empty-dataset-with-index</output-dir>
+ <compilation-unit name="insert-into-empty-dataset">
+ <output-dir compare="Text">insert-into-empty-dataset</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="dml">
@@ -980,6 +985,21 @@
</compilation-unit>
</test-case>
<test-case FilePath="dml">
+ <compilation-unit name="insert-and-scan-dataset">
+ <output-dir compare="Text">insert-and-scan-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-and-scan-dataset-with-index">
+ <output-dir compare="Text">insert-and-scan-dataset-with-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-and-scan-joined-datasets">
+ <output-dir compare="Text">insert-and-scan-joined-datasets</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
<compilation-unit name="insert-into-loaded-dataset-with-index_01">
<output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
</compilation-unit>
@@ -2517,31 +2537,16 @@
</compilation-unit>
</test-case>
<test-case FilePath="numeric">
- <compilation-unit name="unary-minus_double_01">
- <output-dir compare="Text">unary-minus_double_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
<compilation-unit name="unary-minus_double_02">
<output-dir compare="Text">unary-minus_double_02</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="numeric">
- <compilation-unit name="unary-minus_float_01">
- <output-dir compare="Text">unary-minus_float_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
<compilation-unit name="unary-minus_float_02">
<output-dir compare="Text">unary-minus_float_02</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="numeric">
- <compilation-unit name="unary-minus_int_01">
- <output-dir compare="Text">unary-minus_int_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="numeric">
<compilation-unit name="unary-minus_int_02">
<output-dir compare="Text">unary-minus_int_02</output-dir>
</compilation-unit>
@@ -3169,6 +3174,11 @@
<output-dir compare="Text">similarity-jaccard_strings</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="similarity">
+ <compilation-unit name="similarity-jaccard-check_strings_issue628">
+ <output-dir compare="Text">similarity-jaccard-check_strings_issue628</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="spatial">
<test-case FilePath="spatial">
@@ -3862,8 +3872,8 @@
</compilation-unit>
</test-case>
<test-case FilePath="tpch">
- <compilation-unit name="q1_pricing_summary_report_nt">
- <output-dir compare="Text">q1_pricing_summary_report_nt</output-dir>
+ <compilation-unit name="q01_pricing_summary_report_nt">
+ <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="tpch">
@@ -3882,43 +3892,43 @@
</compilation-unit>
</test-case>
<test-case FilePath="tpch">
- <compilation-unit name="q2_minimum_cost_supplier">
- <output-dir compare="Text">q2_minimum_cost_supplier</output-dir>
+ <compilation-unit name="q02_minimum_cost_supplier">
+ <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="tpch">
- <compilation-unit name="q3_shipping_priority_nt">
- <output-dir compare="Text">q3_shipping_priority_nt</output-dir>
+ <compilation-unit name="q03_shipping_priority_nt">
+ <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="tpch">
- <compilation-unit name="q4_order_priority">
- <output-dir compare="Text">q4_order_priority</output-dir>
+ <compilation-unit name="q04_order_priority">
+ <output-dir compare="Text">q04_order_priority</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="tpch">
- <compilation-unit name="q5_local_supplier_volume">
- <output-dir compare="Text">q5_local_supplier_volume</output-dir>
+ <compilation-unit name="q05_local_supplier_volume">
+ <output-dir compare="Text">q05_local_supplier_volume</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="tpch">
- <compilation-unit name="q6_forecast_revenue_change">
- <output-dir compare="Text">q6_forecast_revenue_change</output-dir>
+ <compilation-unit name="q06_forecast_revenue_change">
+ <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="tpch">
- <compilation-unit name="q7_volume_shipping">
- <output-dir compare="Text">q7_volume_shipping</output-dir>
+ <compilation-unit name="q07_volume_shipping">
+ <output-dir compare="Text">q07_volume_shipping</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="tpch">
- <compilation-unit name="q8_national_market_share">
- <output-dir compare="Text">q8_national_market_share</output-dir>
+ <compilation-unit name="q08_national_market_share">
+ <output-dir compare="Text">q08_national_market_share</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="tpch">
- <compilation-unit name="q9_product_type_profit_nt">
- <output-dir compare="Text">q9_product_type_profit_nt</output-dir>
+ <compilation-unit name="q09_product_type_profit_nt">
+ <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
</compilation-unit>
</test-case>
</test-group>
@@ -4248,6 +4258,16 @@
</test-group>
<test-group name="load">
<test-case FilePath="load">
+ <compilation-unit name="csv_01">
+ <output-dir compare="Text">csv_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
+ <compilation-unit name="csv_02">
+ <output-dir compare="Text">csv_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="load">
<compilation-unit name="issue14_query">
<output-dir compare="Text">none</output-dir>
<expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
@@ -4344,6 +4364,36 @@
</test-group>
<test-group name="temporal">
<test-case FilePath="temporal">
+ <compilation-unit name="parse_02">
+ <output-dir compare="Text">parse_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="parse_01">
+ <output-dir compare="Text">parse_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="day_of_week_01">
+ <output-dir compare="Text">day_of_week_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="interval_bin">
+ <output-dir compare="Text">interval_bin</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="interval_bin_gby_0">
+ <output-dir compare="Text">interval_bin_gby_0</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="interval_bin_gby_1">
+ <output-dir compare="Text">interval_bin_gby_1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
<compilation-unit name="accessors">
<output-dir compare="Text">accessors</output-dir>
</compilation-unit>
@@ -4447,4 +4497,11 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="types">
+ <test-case FilePath="types">
+ <compilation-unit name="record01">
+ <output-dir compare="Text">record01</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
</test-suite>
diff --git a/asterix-app/src/test/resources/tpch/queries/asterix/q1_pricing_summary_report.aql b/asterix-app/src/test/resources/tpch/queries/asterix/q01_pricing_summary_report.aql
similarity index 100%
rename from asterix-app/src/test/resources/tpch/queries/asterix/q1_pricing_summary_report.aql
rename to asterix-app/src/test/resources/tpch/queries/asterix/q01_pricing_summary_report.aql
diff --git a/asterix-app/src/test/resources/tpch/queries/asterix/q3_shipping_priority.aql b/asterix-app/src/test/resources/tpch/queries/asterix/q03_shipping_priority.aql
similarity index 100%
rename from asterix-app/src/test/resources/tpch/queries/asterix/q3_shipping_priority.aql
rename to asterix-app/src/test/resources/tpch/queries/asterix/q03_shipping_priority.aql
diff --git a/asterix-app/src/test/resources/tpch/queries/asterix/q5_local_supplier_volume.aql b/asterix-app/src/test/resources/tpch/queries/asterix/q05_local_supplier_volume.aql
similarity index 100%
rename from asterix-app/src/test/resources/tpch/queries/asterix/q5_local_supplier_volume.aql
rename to asterix-app/src/test/resources/tpch/queries/asterix/q05_local_supplier_volume.aql
diff --git a/asterix-app/src/test/resources/tpch/queries/asterix/q9_product_type_profit.aql b/asterix-app/src/test/resources/tpch/queries/asterix/q09_product_type_profit.aql
similarity index 100%
rename from asterix-app/src/test/resources/tpch/queries/asterix/q9_product_type_profit.aql
rename to asterix-app/src/test/resources/tpch/queries/asterix/q09_product_type_profit.aql
diff --git a/asterix-app/src/test/resources/tpch/queries/local/q1_pricing_summary_report.aql b/asterix-app/src/test/resources/tpch/queries/local/q01_pricing_summary_report.aql
similarity index 100%
rename from asterix-app/src/test/resources/tpch/queries/local/q1_pricing_summary_report.aql
rename to asterix-app/src/test/resources/tpch/queries/local/q01_pricing_summary_report.aql
diff --git a/asterix-app/src/test/resources/tpch/queries/local/q3_shipping_priority.aql b/asterix-app/src/test/resources/tpch/queries/local/q03_shipping_priority.aql
similarity index 100%
rename from asterix-app/src/test/resources/tpch/queries/local/q3_shipping_priority.aql
rename to asterix-app/src/test/resources/tpch/queries/local/q03_shipping_priority.aql
diff --git a/asterix-app/src/test/resources/tpch/queries/local/q5_local_supplier_volume.aql b/asterix-app/src/test/resources/tpch/queries/local/q05_local_supplier_volume.aql
similarity index 100%
rename from asterix-app/src/test/resources/tpch/queries/local/q5_local_supplier_volume.aql
rename to asterix-app/src/test/resources/tpch/queries/local/q05_local_supplier_volume.aql
diff --git a/asterix-app/src/test/resources/tpch/queries/local/q9_product_type_profit.aql b/asterix-app/src/test/resources/tpch/queries/local/q09_product_type_profit.aql
similarity index 100%
rename from asterix-app/src/test/resources/tpch/queries/local/q9_product_type_profit.aql
rename to asterix-app/src/test/resources/tpch/queries/local/q09_product_type_profit.aql
diff --git a/asterix-aql/pom.xml b/asterix-aql/pom.xml
index baa2c30..4b83e1c 100644
--- a/asterix-aql/pom.xml
+++ b/asterix-aql/pom.xml
@@ -145,6 +145,11 @@
<version>0.0.3</version>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>xerces</groupId>
+ <artifactId>xerces</artifactId>
+ <version>2.4.0</version>
+ </dependency>
</dependencies>
</project>
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index d71db17..99057f8 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -13,11 +13,12 @@
import java.io.*;
import java.util.List;
import java.util.ArrayList;
-import java.util.Stack;
-
import java.util.Map;
import java.util.HashMap;
import java.util.LinkedHashMap;
+
+import org.apache.xerces.util.IntStack;
+
import edu.uci.ics.asterix.aql.literal.FloatLiteral;
import edu.uci.ics.asterix.aql.literal.DoubleLiteral;
import edu.uci.ics.asterix.aql.literal.FalseLiteral;
@@ -102,6 +103,49 @@
}
return s.substring(1).trim();
}
+
+ private static IRecordFieldDataGen parseFieldDataGen(String hint) throws ParseException {
+ IRecordFieldDataGen rfdg = null;
+ String splits[] = hint.split(" +");
+ if (splits[0].equals(VAL_FILE_HINT)) {
+ File[] valFiles = new File[splits.length - 1];
+ for (int k=1; k<splits.length; k++) {
+ valFiles[k-1] = new File(splits[k]);
+ }
+ rfdg = new FieldValFileDataGen(valFiles);
+ } else if (splits[0].equals(VAL_FILE_SAME_INDEX_HINT)) {
+ rfdg = new FieldValFileSameIndexDataGen(new File(splits[1]), splits[2]);
+ } else if (splits[0].equals(LIST_VAL_FILE_HINT)) {
+ rfdg = new ListValFileDataGen(new File(splits[1]), Integer.parseInt(splits[2]), Integer.parseInt(splits[3]));
+ } else if (splits[0].equals(LIST_HINT)) {
+ rfdg = new ListDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
+ } else if (splits[0].equals(INTERVAL_HINT)) {
+ FieldIntervalDataGen.ValueType vt;
+ if (splits[1].equals("int")) {
+ vt = FieldIntervalDataGen.ValueType.INT;
+ } else if (splits[1].equals("long")) {
+ vt = FieldIntervalDataGen.ValueType.LONG;
+ } else if (splits[1].equals("float")) {
+ vt = FieldIntervalDataGen.ValueType.FLOAT;
+ } else if (splits[1].equals("double")) {
+ vt = FieldIntervalDataGen.ValueType.DOUBLE;
+ } else {
+ throw new ParseException("Unknown type for interval data gen: " + splits[1]);
+ }
+ rfdg = new FieldIntervalDataGen(vt, splits[2], splits[3]);
+ } else if (splits[0].equals(INSERT_RAND_INT_HINT)) {
+ rfdg = new InsertRandIntDataGen(splits[1], splits[2]);
+ } else if (splits[0].equals(DATE_BETWEEN_YEARS_HINT)) {
+ rfdg = new DateBetweenYearsDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
+ } else if (splits[0].equals(DATETIME_BETWEEN_YEARS_HINT)) {
+ rfdg = new DatetimeBetweenYearsDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
+ } else if (splits[0].equals(DATETIME_ADD_RAND_HOURS_HINT)) {
+ rfdg = new DatetimeAddRandHoursDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]), splits[3]);
+ } else if (splits[0].equals(AUTO_HINT)) {
+ rfdg = new AutoDataGen(splits[1]);
+ }
+ return rfdg;
+ }
public AQLParser(String s){
this(new StringReader(s));
@@ -242,7 +286,7 @@
ncNames = new ArrayList<Identifier>();
ncNames.add(new Identifier(tmp));
}
- ( "," tmp = Identifier()
+ ( <COMMA> tmp = Identifier()
{
ncNames.add(new Identifier(tmp));
}
@@ -326,7 +370,7 @@
{
cis.addFieldExpr(fieldExpr);
}
- ) ("," fieldExpr = Identifier()
+ ) (<COMMA> fieldExpr = Identifier()
{
cis.addFieldExpr(fieldExpr);
}
@@ -405,25 +449,12 @@
{
"function" fctName = FunctionName()
ifNotExists = IfNotExists()
- <LEFTPAREN> (<VARIABLE>
- {
- var = new VarIdentifier();
- var.setValue(token.image);
- paramList.add(var);
- getCurrentScope().addNewVarSymbolToScope(var);
- }
- ("," <VARIABLE>
- {
- var = new VarIdentifier();
- var.setValue(token.image);
- paramList.add(var);
- getCurrentScope().addNewVarSymbolToScope(var);
- }
- )*)? <RIGHTPAREN> "{"
- {
- beginPos = token;
- }
- functionBodyExpr = Expression() "}"
+ paramList = ParameterList()
+ <LEFTBRACE>
+ {
+ beginPos = token;
+ }
+ functionBodyExpr = Expression() <RIGHTBRACE>
{
endPos = token;
functionBody = extractFragment(beginPos.beginLine, beginPos.beginColumn, endPos.beginLine, endPos.beginColumn);
@@ -475,7 +506,7 @@
paramList.add(var);
getCurrentScope().addNewVarSymbolToScope(var);
}
- ("," <VARIABLE>
+ (<COMMA> <VARIABLE>
{
var = new VarIdentifier();
var.setValue(token.image);
@@ -554,7 +585,7 @@
{
primaryKeyFields.add(tmp);
}
- ( "," tmp = Identifier()
+ ( <COMMA> tmp = Identifier()
{
primaryKeyFields.add(tmp);
}
@@ -614,7 +645,7 @@
{
}
{
- ( "if" "exists"
+ ( <IF> "exists"
{
return true;
}
@@ -648,7 +679,7 @@
getCurrentScope().addNewVarSymbolToScope(var.getVar());
}
"from" <DATASET> nameComponents = QualifiedName()
- ("where" condition = Expression())?
+ (<WHERE> condition = Expression())?
{
return new DeleteStatement(var, nameComponents.first, nameComponents.second, condition, getVarCounter());
}
@@ -663,13 +694,13 @@
List<UpdateClause> ucs = new ArrayList<UpdateClause>();
}
{
- "update" vars = Variable() "in" target = Expression()
- "where" condition = Expression()
+ "update" vars = Variable() <IN> target = Expression()
+ <WHERE> condition = Expression()
<LEFTPAREN> (uc = UpdateClause()
{
ucs.add(uc);
}
- ("," uc = UpdateClause()
+ (<COMMA> uc = UpdateClause()
{
ucs.add(uc);
}
@@ -691,13 +722,13 @@
UpdateClause elsebranch = null;
}
{
- "set" target = Expression() ":=" value = Expression()
+ "set" target = Expression() <ASSIGN> value = Expression()
| is = InsertStatement()
| ds = DeleteStatement()
| us = UpdateStatement()
- | "if" <LEFTPAREN> condition = Expression() <RIGHTPAREN>
- "then" ifbranch = UpdateClause()
- [LOOKAHEAD(1) "else" elsebranch = UpdateClause()]
+ | <IF> <LEFTPAREN> condition = Expression() <RIGHTPAREN>
+ <THEN> ifbranch = UpdateClause()
+ [LOOKAHEAD(1) <ELSE> elsebranch = UpdateClause()]
{
return new UpdateClause(target, value, is, ds, us, condition, ifbranch, elsebranch);
}
@@ -724,7 +755,7 @@
Pair<Identifier,Identifier> nameComponents = null;
}
{
- "write" "output" "to" nodeName = Identifier() ":" fileName = StringLiteral()
+ "write" "output" "to" nodeName = Identifier() <COLON> fileName = StringLiteral()
( "using" writerClass = StringLiteral() )?
{
return new WriteStatement(new Identifier(nodeName), fileName, writerClass);
@@ -804,7 +835,7 @@
{
configuration.put(keyValuePair.first, keyValuePair.second);
}
- ( "," keyValuePair = KeyValuePair()
+ ( <COMMA> keyValuePair = KeyValuePair()
{
configuration.put(keyValuePair.first, keyValuePair.second);
}
@@ -820,7 +851,7 @@
String value;
}
{
- <LEFTPAREN> key = StringLiteral() "=" value = StringLiteral() <RIGHTPAREN>
+ <LEFTPAREN> key = StringLiteral() <EQ> value = StringLiteral() <RIGHTPAREN>
{
return new Pair<String, String>(key, value);
}
@@ -836,7 +867,7 @@
{
properties.put(property.first, property.second);
}
- ( "," property = Property()
+ ( <COMMA> property = Property()
{
properties.put(property.first, property.second);
}
@@ -852,7 +883,7 @@
String value;
}
{
- key = Identifier() "=" ( value = StringLiteral() | <INTEGER_LITERAL>
+ key = Identifier() <EQ> ( value = StringLiteral() | <INTEGER_LITERAL>
{
try {
value = "" + Long.valueOf(token.image);
@@ -890,7 +921,7 @@
{
( "closed" { recordKind = RecordTypeDefinition.RecordKind.CLOSED; }
| "open" { recordKind = RecordTypeDefinition.RecordKind.OPEN; } )?
- "{"
+ <LEFTBRACE>
{
String hint = getHint(token);
if (hint != null) {
@@ -911,9 +942,9 @@
}
(
RecordField(recType)
- ( "," RecordField(recType) )*
+ ( <COMMA> RecordField(recType) )*
)?
- "}"
+ <RIGHTBRACE>
{
if (recordKind == null) {
recordKind = RecordTypeDefinition.RecordKind.OPEN;
@@ -925,63 +956,20 @@
void RecordField(RecordTypeDefinition recType) throws ParseException:
{
- String fieldName;
- TypeExpression type = null;
- boolean nullable = false;
+ String fieldName;
+ TypeExpression type = null;
+ boolean nullable = false;
}
{
- fieldName = Identifier()
- {
- fieldName = token.image;
- String hint = getHint(token);
- IRecordFieldDataGen rfdg = null;
- if (hint != null) {
- String splits[] = hint.split(" +");
- if (splits[0].equals(VAL_FILE_HINT)) {
- File[] valFiles = new File[splits.length - 1];
- for (int k=1; k<splits.length; k++) {
- valFiles[k-1] = new File(splits[k]);
- }
- rfdg = new FieldValFileDataGen(valFiles);
- } else if (splits[0].equals(VAL_FILE_SAME_INDEX_HINT)) {
- rfdg = new FieldValFileSameIndexDataGen(new File(splits[1]), splits[2]);
- } else if (splits[0].equals(LIST_VAL_FILE_HINT)) {
- rfdg = new ListValFileDataGen(new File(splits[1]), Integer.parseInt(splits[2]), Integer.parseInt(splits[3]));
- } else if (splits[0].equals(LIST_HINT)) {
- rfdg = new ListDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
- } else if (splits[0].equals(INTERVAL_HINT)) {
- FieldIntervalDataGen.ValueType vt;
- if (splits[1].equals("int")) {
- vt = FieldIntervalDataGen.ValueType.INT;
- } else if (splits[1].equals("long")) {
- vt = FieldIntervalDataGen.ValueType.LONG;
- } else if (splits[1].equals("float")) {
- vt = FieldIntervalDataGen.ValueType.FLOAT;
- } else if (splits[1].equals("double")) {
- vt = FieldIntervalDataGen.ValueType.DOUBLE;
- } else {
- throw new ParseException("Unknown type for interval data gen: " + splits[1]);
- }
- rfdg = new FieldIntervalDataGen(vt, splits[2], splits[3]);
- } else if (splits[0].equals(INSERT_RAND_INT_HINT)) {
- rfdg = new InsertRandIntDataGen(splits[1], splits[2]);
- } else if (splits[0].equals(DATE_BETWEEN_YEARS_HINT)) {
- rfdg = new DateBetweenYearsDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
- } else if (splits[0].equals(DATETIME_BETWEEN_YEARS_HINT)) {
- rfdg = new DatetimeBetweenYearsDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
- } else if (splits[0].equals(DATETIME_ADD_RAND_HOURS_HINT)) {
- rfdg = new DatetimeAddRandHoursDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]), splits[3]);
- } else if (splits[0].equals(AUTO_HINT)) {
- rfdg = new AutoDataGen(splits[1]);
- }
- }
- }
- ":"
- ( type = TypeExpr() )
- ("?" { nullable = true; } )?
- {
- recType.addField(fieldName, type, nullable, rfdg);
- }
+ fieldName = Identifier()
+ {
+ String hint = getHint(token);
+ IRecordFieldDataGen rfdg = hint != null ? parseFieldDataGen(hint) : null;
+ }
+ <COLON> type = TypeExpr() (<QUES> { nullable = true; } )?
+ {
+ recType.addField(fieldName, type, nullable, rfdg);
+ }
}
TypeReferenceExpression TypeReference() throws ParseException:
@@ -1000,9 +988,9 @@
TypeExpression type = null;
}
{
- "["
+ <LEFTBRACKET>
( type = TypeExpr() )
- "]"
+ <RIGHTBRACKET>
{
return new OrderedListTypeDefinition(type);
}
@@ -1014,9 +1002,9 @@
TypeExpression type = null;
}
{
- "{{"
+ <LEFTDBLBRACE>
( type = TypeExpr() )
- "}}"
+ <RIGHTDBLBRACE>
{
return new UnorderedListTypeDefinition(type);
}
@@ -1106,7 +1094,7 @@
String second = null;
}
{
- first = Identifier() ("." second = Identifier())?
+ first = Identifier() (<DOT> second = Identifier())?
{
Identifier id1 = null;
Identifier id2 = null;
@@ -1128,7 +1116,7 @@
String third = null;
}
{
- first = Identifier() "." second = Identifier() ("." third = Identifier())?
+ first = Identifier() <DOT> second = Identifier() (<DOT> third = Identifier())?
{
Identifier id1 = null;
Identifier id2 = null;
@@ -1157,7 +1145,7 @@
{
"declare" "function" functionName = Identifier()
paramList = ParameterList()
- "{" funcBody = Expression() "}"
+ <LEFTBRACE> funcBody = Expression() <RIGHTBRACE>
{
signature = new FunctionSignature(defaultDataverse, functionName, paramList.size());
getCurrentScope().addFunctionDescriptor(signature, false);
@@ -1217,7 +1205,7 @@
operand = AndExpr()
(
- "or"
+ <OR>
{
if (op == null) {
op = new OperatorExpr();
@@ -1248,7 +1236,7 @@
operand = RelExpr()
(
- "and"
+ <AND>
{
if (op == null) {
op = new OperatorExpr();
@@ -1291,7 +1279,7 @@
}
(
- LOOKAHEAD(2)( "<" | ">" | "<=" | ">=" | "=" | "!=" |"~=")
+ LOOKAHEAD(2)( <LT> | <GT> | <LE> | <GE> | <EQ> | <NE> |<SIMILAR>)
{
String mhint = getHint(token);
if (mhint != null && mhint.equals(INDEXED_NESTED_LOOP_JOIN_HINT)) {
@@ -1335,7 +1323,7 @@
{
operand = MultExpr()
- ( ("+" | "-")
+ ( (<PLUS> | <MINUS>)
{
if (op == null) {
op = new OperatorExpr();
@@ -1364,7 +1352,7 @@
{
operand = UnionExpr()
- (( "*" | "/" | "%" | <CARET> | "idiv")
+ (( <MUL> | <DIV> | <MOD> | <CARET> | <IDIV>)
{
if (op == null) {
op = new OperatorExpr();
@@ -1392,7 +1380,7 @@
}
{
operand1 = UnaryExpr()
- ("union"
+ (<UNION>
(operand2 = UnaryExpr()) {
if (union == null) {
union = new UnionExpr();
@@ -1411,7 +1399,7 @@
Expression expr = null;
}
{
- (( "+"|"-")
+ ( (<PLUS> | <MINUS>)
{
uexpr = new UnaryExpr();
if("+".equals(token.image))
@@ -1464,7 +1452,7 @@
String ident = null;
}
{
- "." ident = Identifier()
+ <DOT> ident = Identifier()
{
return new Identifier(ident);
}
@@ -1476,7 +1464,7 @@
int idx = -2;
}
{
- "[" ( expr = Expression()
+ <LEFTBRACKET> ( expr = Expression()
{
if(expr.getKind() == Expression.Kind.LITERAL_EXPRESSION)
{
@@ -1492,7 +1480,7 @@
}
- | "?"
+ | <QUES>
{
idx = IndexAccessor.ANY;
// ANY
@@ -1500,7 +1488,7 @@
)
- "]"
+ <RIGHTBRACKET>
{
return idx;
}
@@ -1642,16 +1630,16 @@
}
{
- "["
+ <LEFTBRACKET>
( tmp = Expression()
{
exprList.add(tmp);
}
- ("," tmp = Expression() { exprList.add(tmp); })*
+ (<COMMA> tmp = Expression() { exprList.add(tmp); })*
)?
- "]"
+ <RIGHTBRACKET>
{
expr.setExprList(exprList);
@@ -1668,11 +1656,11 @@
}
{
- "{{" ( tmp = Expression()
+ <LEFTDBLBRACE> ( tmp = Expression()
{
exprList.add(tmp);
}
- ("," tmp = Expression() { exprList.add(tmp); })*)? "}}"
+ (<COMMA> tmp = Expression() { exprList.add(tmp); })*)? <RIGHTDBLBRACE>
{
expr.setExprList(exprList);
return expr;
@@ -1686,11 +1674,11 @@
List<FieldBinding> fbList = new ArrayList<FieldBinding>();
}
{
- "{" (tmp = FieldBinding()
+ <LEFTBRACE> (tmp = FieldBinding()
{
fbList.add(tmp);
}
- ("," tmp = FieldBinding() { fbList.add(tmp); })*)? "}"
+ (<COMMA> tmp = FieldBinding() { fbList.add(tmp); })*)? <RIGHTBRACE>
{
expr.setFbList(fbList);
return expr;
@@ -1703,7 +1691,7 @@
Expression left, right;
}
{
- left = Expression() ":" right = Expression()
+ left = Expression() <COLON> right = Expression()
{
fb.setLeftExpr(left);
fb.setRightExpr(right);
@@ -1731,7 +1719,7 @@
argList.add(tmp);
arity ++;
}
- ("," tmp = Expression()
+ (<COMMA> tmp = Expression()
{
argList.add(tmp);
arity++;
@@ -1766,7 +1754,7 @@
{
funcName = token.image;
}
- ( ( arg1 = Identifier() ( "." arg2 = Identifier() )? )
+ ( ( arg1 = Identifier() ( <DOT> arg2 = Identifier() )? )
{
String name = arg2 == null ? arg1 : arg1 + "." + arg2;
LiteralExpr ds = new LiteralExpr();
@@ -1805,7 +1793,7 @@
IfExpr ifExpr = new IfExpr();
}
{
- "if" <LEFTPAREN> condExpr = Expression() <RIGHTPAREN> "then" thenExpr = Expression() "else" elseExpr = Expression()
+ <IF> <LEFTPAREN> condExpr = Expression() <RIGHTPAREN> <THEN> thenExpr = Expression() <ELSE> elseExpr = Expression()
{
ifExpr.setCondExpr(condExpr);
@@ -1825,7 +1813,7 @@
}
{
(tmp = ForClause() {clauseList.add(tmp);} | tmp = LetClause() {clauseList.add(tmp);})
- (tmp = Clause() {clauseList.add(tmp);})* "return" returnExpr = Expression()
+ (tmp = Clause() {clauseList.add(tmp);})* <RETURN> returnExpr = Expression()
{
flworg.setClauseList(clauseList);
@@ -1863,7 +1851,7 @@
extendCurrentScope();
}
{
- "for" varExp = Variable() ("at" varPos = Variable())? "in" ( inExp = Expression() )
+ <FOR> varExp = Variable() (<AT> varPos = Variable())? <IN> ( inExp = Expression() )
{
fc.setVarExpr(varExp);
getCurrentScope().addNewVarSymbolToScope(varExp.getVar());
@@ -1884,7 +1872,7 @@
extendCurrentScope();
}
{
- "let" varExp = Variable() ":=" beExp = Expression()
+ <LET> varExp = Variable() <ASSIGN> beExp = Expression()
{
getCurrentScope().addNewVarSymbolToScope(varExp.getVar());
lc.setVarExpr(varExp);
@@ -1899,7 +1887,7 @@
Expression whereExpr;
}
{
- "where" whereExpr = Expression()
+ <WHERE> whereExpr = Expression()
{
wc.setWhereExpr(whereExpr);
return wc;
@@ -1916,7 +1904,7 @@
}
{
(
- "order"
+ <ORDER>
{
String hint = getHint(token);
if (hint != null && hint.startsWith(INMEMORY_HINT)) {
@@ -1927,24 +1915,24 @@
oc.setNumTuples(numTuples);
}
}
- "by" orderbyExpr = Expression()
+ <BY> orderbyExpr = Expression()
{
orderbyList.add(orderbyExpr);
OrderbyClause.OrderModifier modif = OrderbyClause.OrderModifier.ASC;
}
- ( ("asc" { modif = OrderbyClause.OrderModifier.ASC; })
- | ("desc" { modif = OrderbyClause.OrderModifier.DESC; }))?
+ ( (<ASC> { modif = OrderbyClause.OrderModifier.ASC; })
+ | (<DESC> { modif = OrderbyClause.OrderModifier.DESC; }))?
{
modifierList.add(modif);
}
- ("," orderbyExpr = Expression()
+ (<COMMA> orderbyExpr = Expression()
{
orderbyList.add(orderbyExpr);
modif = OrderbyClause.OrderModifier.ASC;
}
- ( ("asc" { modif = OrderbyClause.OrderModifier.ASC; })
- | ("desc" { modif = OrderbyClause.OrderModifier.DESC; }))?
+ ( (<ASC> { modif = OrderbyClause.OrderModifier.ASC; })
+ | (<DESC> { modif = OrderbyClause.OrderModifier.DESC; }))?
{
modifierList.add(modif);
}
@@ -1974,39 +1962,39 @@
Scope newScope = extendCurrentScopeNoPush(true);
// extendCurrentScope(true);
}
- "group"
+ <GROUP>
{
String hint = getHint(token);
if (hint != null && hint.equals(HASH_GROUP_BY_HINT)) {
gbc.setHashGroupByHint(true);
}
}
- "by" (LOOKAHEAD(2) var = Variable()
+ <BY> (LOOKAHEAD(2) var = Variable()
{
newScope.addNewVarSymbolToScope(var.getVar());
- } ":=")?
+ } <ASSIGN>)?
expr = Expression()
{
GbyVariableExpressionPair pair1 = new GbyVariableExpressionPair(var, expr);
vePairList.add(pair1);
}
- ("," ( LOOKAHEAD(2) var = Variable()
+ (<COMMA> ( LOOKAHEAD(2) var = Variable()
{
newScope.addNewVarSymbolToScope(var.getVar());
- } ":=")?
+ } <ASSIGN>)?
expr = Expression()
{
GbyVariableExpressionPair pair2 = new GbyVariableExpressionPair(var, expr);
vePairList.add(pair2);
}
)*
- ("decor" decorVar = Variable() ":=" decorExpr = Expression()
+ (<DECOR> decorVar = Variable() <ASSIGN> decorExpr = Expression()
{
newScope.addNewVarSymbolToScope(decorVar.getVar());
GbyVariableExpressionPair pair3 = new GbyVariableExpressionPair(decorVar, decorExpr);
decorPairList.add(pair3);
}
- ("," "decor" decorVar = Variable() ":=" decorExpr = Expression()
+ (<COMMA> <DECOR> decorVar = Variable() <ASSIGN> decorExpr = Expression()
{
newScope.addNewVarSymbolToScope(decorVar.getVar());
GbyVariableExpressionPair pair4 = new GbyVariableExpressionPair(decorVar, decorExpr);
@@ -2014,14 +2002,14 @@
}
)*
)?
- "with" withVar = VariableRef()
+ <WITH> withVar = VariableRef()
{
if(withVar.getIsNewVar()==true)
throw new ParseException("can't find variable " + withVar.getVar());
withVarList.add(withVar);
newScope.addNewVarSymbolToScope(withVar.getVar());
}
- ("," withVar = VariableRef()
+ (<COMMA> withVar = VariableRef()
{
if(withVar.getIsNewVar()==true)
throw new ParseException("can't find variable " + withVar.getVar());
@@ -2045,8 +2033,8 @@
pushForbiddenScope(getCurrentScope());
}
{
- "limit" expr = Expression() { lc.setLimitExpr(expr); }
- ("offset" expr = Expression() { lc.setOffset(expr); })?
+ <LIMIT> expr = Expression() { lc.setLimitExpr(expr); }
+ (<OFFSET> expr = Expression() { lc.setOffset(expr); })?
{
popForbiddenScope();
@@ -2060,11 +2048,11 @@
Expression expr;
}
{
- "distinct" "by" expr = Expression()
+ <DISTINCT> <BY> expr = Expression()
{
exprs.add(expr);
}
- ("," expr = Expression()
+ (<COMMA> expr = Expression()
{
exprs.add(expr);
}
@@ -2088,23 +2076,23 @@
createNewScope();
}
- ( ("some" { qc.setQuantifier(QuantifiedExpression.Quantifier.SOME); })
- | ("every" { qc.setQuantifier(QuantifiedExpression.Quantifier.EVERY); }))
- var = Variable() "in" inExpr = Expression()
+ ( (<SOME> { qc.setQuantifier(QuantifiedExpression.Quantifier.SOME); })
+ | (<EVERY> { qc.setQuantifier(QuantifiedExpression.Quantifier.EVERY); }))
+ var = Variable() <IN> inExpr = Expression()
{
pair = new QuantifiedPair(var, inExpr);
getCurrentScope().addNewVarSymbolToScope(var.getVar());
quantifiedList.add(pair);
}
(
- "," var = Variable() "in" inExpr = Expression()
+ <COMMA> var = Variable() <IN> inExpr = Expression()
{
pair = new QuantifiedPair(var, inExpr);
getCurrentScope().addNewVarSymbolToScope(var.getVar());
quantifiedList.add(pair);
}
)*
- "satisfies" satisfiesExpr = Expression()
+ <SATISFIES> satisfiesExpr = Expression()
{
qc.setSatisfiesExpr(satisfiesExpr);
qc.setQuantifiedList(quantifiedList);
@@ -2115,155 +2103,212 @@
TOKEN_MGR_DECLS:
{
- public int commentDepth = 0;
+ public int commentDepth = 0;
+ public IntStack lexerStateStack = new IntStack();
+
+ public void pushState() {
+ lexerStateStack.push( curLexState );
+ }
+
+ public void popState() {
+ if (lexerStateStack.size() > 0) {
+ SwitchTo( lexerStateStack.pop() );
+ } else {
+ throw new RuntimeException();
+ }
+ }
+}
+
+<DEFAULT,IN_DBL_BRACE>
+TOKEN :
+{
+ <ASC : "asc">
+ | <AT : "at">
+ | <BY : "by">
+ | <DATASET : "dataset">
+ | <DECOR : "decor">
+ | <DESC : "desc">
+ | <DISTINCT : "distinct">
+ | <ELSE : "else">
+ | <EVERY : "every">
+ | <FOR : "for">
+ | <GROUP : "group">
+ | <IF : "if">
+ | <IN : "in">
+ | <LET : "let">
+ | <LIMIT : "limit">
+ | <OFFSET : "offset">
+ | <ORDER : "order">
+ | <RETURN : "return">
+ | <SATISFIES : "satisfies">
+ | <SOME : "some">
+ | <THEN : "then">
+ | <UNION : "union">
+ | <WHERE : "where">
+ | <WITH : "with">
+}
+
+<DEFAULT,IN_DBL_BRACE>
+TOKEN :
+{
+ <CARET : "^">
+ | <DIV : "/">
+ | <IDIV : "idiv">
+ | <MINUS : "-">
+ | <MOD : "%">
+ | <MUL : "*">
+ | <PLUS : "+">
+
+ | <LEFTPAREN : "(">
+ | <RIGHTPAREN : ")">
+ | <LEFTBRACKET : "[">
+ | <RIGHTBRACKET : "]">
+
+ | <COLON : ":">
+ | <COMMA : ",">
+ | <DOT : ".">
+ | <QUES : "?">
+
+ | <LT : "<">
+ | <GT : ">">
+ | <LE : "<=">
+ | <GE : ">=">
+ | <EQ : "=">
+ | <NE : "!=">
+ | <SIMILAR : "~=">
+ | <ASSIGN : ":=">
+
+ | <AND : "and">
+ | <OR : "or">
+}
+
+<DEFAULT,IN_DBL_BRACE>
+TOKEN :
+{
+ <LEFTBRACE : "{"> { pushState(); } : DEFAULT
}
<DEFAULT>
TOKEN :
{
- <CARET : "^" >
+ <RIGHTBRACE : "}"> { popState(); }
}
-<DEFAULT>
+<DEFAULT,IN_DBL_BRACE>
TOKEN :
{
- <DATASET : "dataset" >
+ <LEFTDBLBRACE : "{{"> { pushState(); } : IN_DBL_BRACE
}
-<DEFAULT>
+<IN_DBL_BRACE>
TOKEN :
{
- <LEFTPAREN : "(" >
+ <RIGHTDBLBRACE : "}}"> { popState(); }
}
-<DEFAULT>
+<DEFAULT,IN_DBL_BRACE>
TOKEN :
{
- <RIGHTPAREN : ")" >
+ <INTEGER_LITERAL : (<DIGIT>)+ >
}
-
-<DEFAULT>
+<DEFAULT,IN_DBL_BRACE>
TOKEN :
{
- <INTEGER_LITERAL : (<DIGIT>)+ >
+ <NULL : "null">
+ | <TRUE : "true">
+ | <FALSE : "false">
}
-
-<DEFAULT>
+<DEFAULT,IN_DBL_BRACE>
TOKEN :
{
- <NULL : "null">
+ <#DIGIT : ["0" - "9"]>
}
-<DEFAULT>
-TOKEN :
-{
- <TRUE : "true">
-}
-
-<DEFAULT>
-TOKEN :
-{
- <FALSE : "false">
-}
-
-<DEFAULT>
-TOKEN :
-{
- <#DIGIT : ["0" - "9"]>
-}
-
-
+<DEFAULT,IN_DBL_BRACE>
TOKEN:
{
- < DOUBLE_LITERAL: <INTEGER>
- | <INTEGER> ( "." <INTEGER> )?
- | "." <INTEGER>
- >
- |
- < FLOAT_LITERAL: <INTEGER> ( "f" | "F" )
- | <INTEGER> ( "." <INTEGER> ( "f" | "F" ) )?
- | "." <INTEGER> ( "f" | "F" )
- >
- |
- <INTEGER : (<DIGIT>)+ >
+ < DOUBLE_LITERAL: <DIGITS>
+ | <DIGITS> ( "." <DIGITS> )?
+ | "." <DIGITS>
+ >
+ | < FLOAT_LITERAL: <DIGITS> ( "f" | "F" )
+ | <DIGITS> ( "." <DIGITS> ( "f" | "F" ) )?
+ | "." <DIGITS> ( "f" | "F" )
+ >
+ | <DIGITS : (<DIGIT>)+ >
}
-<DEFAULT>
+<DEFAULT,IN_DBL_BRACE>
TOKEN :
{
- <#LETTER : ["A" - "Z", "a" - "z"]>
+ <#LETTER : ["A" - "Z", "a" - "z"]>
+ | <SPECIALCHARS : ["$", "_", "-"]>
}
-<DEFAULT>
+<DEFAULT,IN_DBL_BRACE>
TOKEN :
{
- <SPECIALCHARS : ["$", "_", "-"] >
+ <STRING_LITERAL : ("\"" (<EscapeQuot> | ~["\""])* "\"") | ("\'"(<EscapeApos> | ~["\'"])* "\'")>
+ | < #EscapeQuot: "\\\"" >
+ | < #EscapeApos: "\\\'" >
}
-<DEFAULT>
+<DEFAULT,IN_DBL_BRACE>
TOKEN :
{
- <STRING_LITERAL : ("\"" (<EscapeQuot> | ~["\""])* "\"") | ("\'"(<EscapeApos> | ~["\'"])* "\'")>
- |
- < #EscapeQuot: "\\\"" >
- |
- < #EscapeApos: "\\\'" >
+ <IDENTIFIER : <LETTER> (<LETTER> | <DIGIT> | <SPECIALCHARS>)*>
}
-<DEFAULT>
+<DEFAULT,IN_DBL_BRACE>
TOKEN :
{
- <IDENTIFIER : (<LETTER>)+ (<LETTER> | <DIGIT> | <SPECIALCHARS>)*>
+ <VARIABLE : "$" <LETTER> (<LETTER> | <DIGIT> | "_")*>
}
-
-<DEFAULT>
-TOKEN :
-{
- <VARIABLE : "$" (<LETTER>)+ (<LETTER> | <DIGIT> | "_")* >
-}
-
+<DEFAULT,IN_DBL_BRACE>
SKIP:
{
" "
-| "\t"
-| "\r"
-| "\n"
+ | "\t"
+ | "\r"
+ | "\n"
}
+<DEFAULT,IN_DBL_BRACE>
SKIP:
{
- <"//" (~["\n"])* "\n">
+ <"//" (~["\n"])* "\n">
}
+<DEFAULT,IN_DBL_BRACE>
SKIP:
{
- <"//" (~["\n","\r"])* ("\n"|"\r"|"\r\n")?>
+ <"//" (~["\n","\r"])* ("\n"|"\r"|"\r\n")?>
}
-
+<DEFAULT,IN_DBL_BRACE>
SKIP:
{
- <"/*"> {commentDepth=1;}: INSIDE_COMMENT
+ <"/*"> { pushState(); } : INSIDE_COMMENT
}
<INSIDE_COMMENT>
SPECIAL_TOKEN:
{
- <"+"(" ")*(~["*"])*>
+ <"+"(" ")*(~["*"])*>
}
<INSIDE_COMMENT>
SKIP:
{
- <"/*"> {commentDepth++;}
+ <"/*"> { pushState(); }
}
<INSIDE_COMMENT>
SKIP:
{
- <"*/"> {commentDepth--; if (commentDepth == 0) SwitchTo(DEFAULT);}
-| <~[]>
+ <"*/"> { popState(); }
+ | <~[]>
}
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q01_pricing_summary_report_nt.3.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.3.query.aql
copy to asterix-benchmarks/src/main/resources/tpc-h/queries/q01_pricing_summary_report_nt.3.query.aql
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q02_minimum_cost_supplier.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q02_minimum_cost_supplier.3.query.aql
new file mode 100644
index 0000000..f9b460b
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q02_minimum_cost_supplier.3.query.aql
@@ -0,0 +1,117 @@
+use dataverse tpch;
+
+declare function tmp1() {
+ for $p in dataset('Part')
+ for $pssrn in (
+ for $ps in dataset('Partsupp')
+ for $srn in (
+ for $s in dataset('Supplier')
+ for $rn in (
+ for $r in dataset('Region')
+ for $n in dataset('Nation')
+ where $n.n_regionkey = $r.r_regionkey and $r.r_name = 'EUROPE'
+ return {
+ "n_nationkey": $n.n_nationkey,
+ "n_name": $n.n_name
+ }
+ )
+ where $s.s_nationkey = $rn.n_nationkey
+ return {
+ "s_suppkey": $s.s_suppkey,
+ "n_name": $rn.n_name,
+ "s_name": $s.s_name,
+ "s_acctbal": $s.s_acctbal,
+ "s_address": $s.s_address,
+ "s_phone": $s.s_phone,
+ "s_comment": $s.s_comment
+ }
+ )
+ where $srn.s_suppkey = $ps.ps_suppkey
+ return {
+ "n_name": $srn.n_name,
+ "p_partkey": $ps.ps_partkey,
+ "ps_supplycost": $ps.ps_supplycost,
+ "s_name": $srn.s_name,
+ "s_acctbal": $srn.s_acctbal,
+ "s_address": $srn.s_address,
+ "s_phone": $srn.s_phone,
+ "s_comment": $srn.s_comment
+ }
+ )
+ where $p.p_partkey = $pssrn.p_partkey and like($p.p_type, '%BRASS') and $p.p_size = 15
+ return {
+ "s_acctbal": $pssrn.s_acctbal,
+ "s_name": $pssrn.s_name,
+ "n_name": $pssrn.n_name,
+ "p_partkey": $p.p_partkey,
+ "ps_supplycost": $pssrn.ps_supplycost,
+ "p_mfgr": $p.p_mfgr,
+ "s_address": $pssrn.s_address,
+ "s_phone": $pssrn.s_phone,
+ "s_comment": $pssrn.s_comment
+ }
+}
+
+declare function tmp2(){
+ for $p in dataset('Part')
+ for $pssrn in (
+ for $ps in dataset('Partsupp')
+ for $srn in (
+ for $s in dataset('Supplier')
+ for $rn in (
+ for $r in dataset('Region')
+ for $n in dataset('Nation')
+ where $n.n_regionkey = $r.r_regionkey and $r.r_name = 'EUROPE'
+ return {
+ "n_nationkey": $n.n_nationkey,
+ "n_name": $n.n_name
+ }
+ )
+ where $s.s_nationkey = $rn.n_nationkey
+ return {
+ "s_suppkey": $s.s_suppkey,
+ "n_name": $rn.n_name,
+ "s_name": $s.s_name,
+ "s_acctbal": $s.s_acctbal,
+ "s_address": $s.s_address,
+ "s_phone": $s.s_phone,
+ "s_comment": $s.s_comment
+ }
+ )
+ where $srn.s_suppkey = $ps.ps_suppkey
+ return {
+ "n_name": $srn.n_name,
+ "p_partkey": $ps.ps_partkey,
+ "ps_supplycost": $ps.ps_supplycost,
+ "s_name": $srn.s_name,
+ "s_acctbal": $srn.s_acctbal,
+ "s_address": $srn.s_address,
+ "s_phone": $srn.s_phone,
+ "s_comment": $srn.s_comment
+ }
+ )
+ where $p.p_partkey = $pssrn.p_partkey and like($p.p_type, '%BRASS') and $p.p_size = 15
+ /*+ hash*/
+ group by $p_partkey := $pssrn.p_partkey with $pssrn
+ return {
+ "p_partkey": $p_partkey,
+ "ps_min_supplycost": min(for $i in $pssrn return $i.ps_supplycost)
+ }
+}
+
+for $t2 in tmp2()
+for $t1 in tmp1()
+where $t1.p_partkey = $t2.p_partkey and $t1.ps_supplycost = $t2.ps_min_supplycost
+order by $t1.s_acctbal desc, $t1.n_name, $t1.s_name, $t1.p_partkey
+limit 100
+return
+{
+ "s_acctbal": $t1.s_acctbal,
+ "s_name": $t1.s_name,
+ "n_name": $t1.n_name,
+ "p_partkey": $t1.p_partkey,
+ "p_mfgr": $t1.p_mfgr,
+ "s_address": $t1.s_address,
+ "s_phone": $t1.s_phone,
+ "s_comment": $t1.s_comment
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q3_shipping_priority_nt/q3_shipping_priority_nt.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q03_shipping_priority_nt.3.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/tpch/q3_shipping_priority_nt/q3_shipping_priority_nt.3.query.aql
copy to asterix-benchmarks/src/main/resources/tpc-h/queries/q03_shipping_priority_nt.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q4_order_priority/q4_order_priority.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q04_order_priority.3.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/tpch/q4_order_priority/q4_order_priority.3.query.aql
copy to asterix-benchmarks/src/main/resources/tpc-h/queries/q04_order_priority.3.query.aql
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q05_local_supplier_volume.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q05_local_supplier_volume.3.query.aql
new file mode 100644
index 0000000..342a85a
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q05_local_supplier_volume.3.query.aql
@@ -0,0 +1,57 @@
+use dataverse tpch;
+
+for $c in dataset('Customer')
+for $o1 in (
+ for $o in dataset('Orders')
+ for $l1 in (
+ for $l in dataset('LineItem')
+ for $s1 in (
+ for $s in dataset('Supplier')
+ for $n1 in (
+ for $n in dataset('Nation')
+ for $r in dataset('Region')
+ where $n.n_regionkey = $r.r_regionkey
+ and $r.r_name = 'ASIA'
+ return {
+ "n_name": $n.n_name,
+ "n_nationkey": $n.n_nationkey
+ }
+ )
+ where $s.s_nationkey = $n1.n_nationkey
+ return {
+ "n_name": $n1.n_name,
+ "s_suppkey": $s.s_suppkey,
+ "s_nationkey": $s.s_nationkey
+ }
+ )
+ where $l.l_suppkey = $s1.s_suppkey
+ return {
+ "n_name": $s1.n_name,
+ "l_extendedprice": $l.l_extendedprice,
+ "l_discount": $l.l_discount,
+ "l_orderkey": $l.l_orderkey,
+ "s_nationkey": $s1.s_nationkey
+ }
+ )
+ where $l1.l_orderkey = $o.o_orderkey and $o.o_orderdate >= '1994-01-01' and $o.o_orderdate < '1995-01-01'
+ return {
+ "n_name": $l1.n_name,
+ "l_extendedprice": $l1.l_extendedprice,
+ "l_discount": $l1.l_discount,
+ "s_nationkey": $l1.s_nationkey,
+ "o_custkey": $o.o_custkey
+ }
+)
+where $c.c_nationkey = $o1.s_nationkey and $c.c_custkey = $o1.o_custkey
+/*+ hash*/
+group by $n_name := $o1.n_name with $o1
+let $revenue := sum (
+ for $i in $o1
+ return
+ $i.l_extendedprice * (1 - $i.l_discount)
+)
+order by $revenue desc
+return {
+ "n_name": $n_name,
+ "revenue": $revenue
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q06_forecast_revenue_change.3.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/tpch/q6_forecast_revenue_change/q6_forecast_revenue_change.3.query.aql
copy to asterix-benchmarks/src/main/resources/tpc-h/queries/q06_forecast_revenue_change.3.query.aql
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q07_volume_shipping.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q07_volume_shipping.3.query.aql
new file mode 100644
index 0000000..68e6d41
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q07_volume_shipping.3.query.aql
@@ -0,0 +1,63 @@
+use dataverse tpch;
+
+declare function q7_volume_shipping_tmp() {
+ for $n1 in dataset('Nation')
+ for $n2 in dataset('Nation')
+ where ($n1.n_name='FRANCE' and $n2.n_name='GERMANY') or ($n1.n_name='GERMANY' and $n2.n_name='FRANCE')
+ return {
+ "supp_nation": $n1.n_name,
+ "cust_nation": $n2.n_name,
+ "s_nationkey": $n1.n_nationkey,
+ "c_nationkey": $n2.n_nationkey
+ }
+}
+
+for $locs in (
+ for $loc in (
+ for $lo in (
+ for $l in dataset('LineItem')
+ for $o in dataset('Orders')
+ where $o.o_orderkey = $l.l_orderkey and $l.l_shipdate >= '1995-01-01'
+ and $l.l_shipdate <= '1996-12-31'
+ return {
+ "l_shipdate": $l.l_shipdate,
+ "l_extendedprice": $l.l_extendedprice,
+ "l_discount": $l.l_discount,
+ "l_suppkey": $l.l_suppkey,
+ "o_custkey": $o.o_custkey
+ }
+ )
+ for $c in dataset('Customer')
+ where $c.c_custkey = $lo.o_custkey
+ return {
+ "l_shipdate": $lo.l_shipdate,
+ "l_extendedprice": $lo.l_extendedprice,
+ "l_discount": $lo.l_discount,
+ "l_suppkey": $lo.l_suppkey,
+ "c_nationkey": $c.c_nationkey
+ }
+ )
+ for $s in dataset('Supplier')
+ where $s.s_suppkey = $loc.l_suppkey
+ return {
+ "l_shipdate": $loc.l_shipdate,
+ "l_extendedprice": $loc.l_extendedprice,
+ "l_discount": $loc.l_discount,
+ "c_nationkey": $loc.c_nationkey,
+ "s_nationkey": $s.s_nationkey
+ }
+)
+for $t in q7_volume_shipping_tmp()
+where $locs.c_nationkey = $t.c_nationkey
+ and $locs.s_nationkey = $t.s_nationkey
+let $l_year0 := get-year($locs.l_shipdate)
+group by $supp_nation := $t.supp_nation, $cust_nation := $t.cust_nation, $l_year := $l_year0
+with $locs
+let $revenue := sum(for $i in $locs return $i.l_extendedprice * (1 - $i.l_discount))
+order by $supp_nation, $cust_nation, $l_year
+return {
+ "supp_nation": $supp_nation,
+ "cust_nation": $cust_nation,
+ "l_year": $l_year,
+ "revenue": $revenue
+}
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q08_national_market_share.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q08_national_market_share.3.query.aql
new file mode 100644
index 0000000..7a8a2a1
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q08_national_market_share.3.query.aql
@@ -0,0 +1,73 @@
+use dataverse tpch;
+
+for $t in (
+ for $slnrcop in (
+ for $s in dataset("Supplier")
+ for $lnrcop in (
+ for $lnrco in (
+ for $l in dataset('LineItem')
+ for $nrco in (
+ for $o in dataset('Orders')
+ for $nrc in (
+ for $c in dataset('Customer')
+ for $nr in (
+ for $n1 in dataset('Nation')
+ for $r1 in dataset('Region')
+ where $n1.n_regionkey = $r1.r_regionkey and $r1.r_name = 'AMERICA'
+ return { "n_nationkey": $n1.n_nationkey }
+ )
+ where $c.c_nationkey = $nr.n_nationkey
+ return { "c_custkey": $c.c_custkey }
+ )
+ where $nrc.c_custkey = $o.o_custkey
+ return {
+ "o_orderdate" : $o.o_orderdate,
+ "o_orderkey": $o.o_orderkey
+ }
+ )
+ where $l.l_orderkey = $nrco.o_orderkey
+ and $nrco.o_orderdate >= '1995-01-01'
+ and $nrco.o_orderdate <= '1996-12-31'
+ return {
+ "o_orderdate": $nrco.o_orderdate,
+ "l_partkey": $l.l_partkey,
+ "l_discount": $l.l_discount,
+ "l_extendedprice": $l.l_extendedprice,
+ "l_suppkey": $l.l_suppkey
+ }
+ )
+ for $p in dataset("Part")
+ where $p.p_partkey = $lnrco.l_partkey and $p.p_type = 'ECONOMY ANODIZED STEEL'
+ return {
+ "o_orderdate": $lnrco.o_orderdate,
+ "l_discount": $lnrco.l_discount,
+ "l_extendedprice": $lnrco.l_extendedprice,
+ "l_suppkey": $lnrco.l_suppkey
+ }
+ )
+ where $s.s_suppkey = $lnrcop.l_suppkey
+ return {
+ "o_orderdate": $lnrcop.o_orderdate,
+ "l_discount": $lnrcop.l_discount,
+ "l_extendedprice": $lnrcop.l_extendedprice,
+ "l_suppkey": $lnrcop.l_suppkey,
+ "s_nationkey": $s.s_nationkey
+ }
+ )
+ for $n2 in dataset('Nation')
+ where $slnrcop.s_nationkey = $n2.n_nationkey
+ let $o_year := get-year($slnrcop.o_orderdate)
+ return {
+ "year": $o_year,
+ "revenue": $slnrcop.l_extendedprice *(1-$slnrcop.l_discount),
+ "s_name": $n2.n_name
+ }
+)
+group by $year := $t.year with $t
+order by $year
+return {
+ "year": $year,
+ "mkt_share": sum(for $i in $t return switch-case($i.s_name='BRAZIL', true, $i.revenue, false, 0.0))/
+ sum(for $i in $t return $i.revenue)
+}
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q09_product_type_profit_nt.3.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/tpch/q9_product_type_profit_nt/q9_product_type_profit_nt.3.query.aql
copy to asterix-benchmarks/src/main/resources/tpc-h/queries/q09_product_type_profit_nt.3.query.aql
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q10_returned_item.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q10_returned_item.3.query.aql
new file mode 100644
index 0000000..19a2b44
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q10_returned_item.3.query.aql
@@ -0,0 +1,55 @@
+use dataverse tpch;
+
+for $locn in (
+ for $l in dataset('LineItem')
+ for $ocn in (
+ for $o in dataset('Orders')
+ for $c in dataset('Customer')
+ where $c.c_custkey = $o.o_custkey and $o.o_orderdate >= '1993-10-01'
+ and $o.o_orderdate < '1994-01-01'
+ for $n in dataset('Nation')
+ where $c.c_nationkey = $n.n_nationkey
+ return {
+ "c_custkey": $c.c_custkey,
+ "c_name": $c.c_name,
+ "c_acctbal": $c.c_acctbal,
+ "n_name": $n.n_name,
+ "c_address": $c.c_address,
+ "c_phone": $c.c_phone,
+ "c_comment": $c.c_comment,
+ "o_orderkey": $o.o_orderkey
+ }
+ )
+ where $l.l_orderkey = $ocn.o_orderkey and $l.l_returnflag = 'R'
+ return {
+ "c_custkey": $ocn.c_custkey,
+ "c_name": $ocn.c_name,
+ "c_acctbal": $ocn.c_acctbal,
+ "n_name": $ocn.n_name,
+ "c_address": $ocn.c_address,
+ "c_phone": $ocn.c_phone,
+ "c_comment": $ocn.c_comment,
+ "l_extendedprice": $l.l_extendedprice,
+ "l_discount": $l.l_discount
+ }
+)
+group by $c_custkey:=$locn.c_custkey,
+ $c_name:=$locn.c_name,
+ $c_acctbal:=$locn.c_acctbal, $c_phone:=$locn.c_phone,
+ $n_name:=$locn.n_name, $c_address:=$locn.c_address, $c_comment:=$locn.c_comment
+ with $locn
+let $revenue := sum(for $i in $locn return $i.l_extendedprice * (1 - $i.l_discount))
+order by $revenue desc
+limit 20
+return {
+ "c_custkey": $c_custkey,
+ "c_name": $c_name,
+ "revenue": $revenue,
+ "c_acctbal": $c_acctbal,
+ "n_name": $n_name,
+ "c_address": $c_address,
+ "c_phone": $c_phone,
+ "c_comment": $c_comment
+}
+
+
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q11_important_stock.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q11_important_stock.3.query.aql
new file mode 100644
index 0000000..38286e6
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q11_important_stock.3.query.aql
@@ -0,0 +1,36 @@
+use dataverse tpch;
+
+let $sum := sum (
+ for $ps in dataset('Partsupp')
+ for $sn in (
+ for $s in dataset('Supplier')
+ for $n in dataset('Nation')
+ where $s.s_nationkey = $n.n_nationkey
+ and $n.n_name = 'GERMANY'
+ return { "s_suppkey": $s.s_suppkey }
+ )
+ where $ps.ps_suppkey = $sn.s_suppkey
+ return $ps.ps_supplycost * $ps.ps_availqty
+)
+for $t1 in (
+ for $ps in dataset('Partsupp')
+ for $sn in (
+ for $s in dataset('Supplier')
+ for $n in dataset('Nation')
+ where $s.s_nationkey = $n.n_nationkey
+ and $n.n_name = 'GERMANY'
+ return { "s_suppkey": $s.s_suppkey }
+ )
+ where $ps.ps_suppkey = $sn.s_suppkey
+ group by $ps_partkey := $ps.ps_partkey with $ps
+ return {
+ "ps_partkey": $ps_partkey,
+ "part_value": sum(for $i in $ps return $i.ps_supplycost * $i.ps_availqty)
+ }
+)
+where $t1.part_value > $sum * 0.0001
+order by $t1.part_value desc
+return {
+ "partkey": $t1.ps_partkey,
+ "part_value": $t1.part_value
+}
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q12_shipping.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q12_shipping.3.query.aql
new file mode 100644
index 0000000..c836a48
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q12_shipping.3.query.aql
@@ -0,0 +1,27 @@
+use dataverse tpch;
+
+for $l in dataset('LineItem')
+for $o in dataset('Orders')
+where $o.o_orderkey = $l.l_orderkey
+ and $l.l_commitdate < $l.l_receiptdate
+ and $l.l_shipdate < $l.l_commitdate
+ and $l.l_receiptdate >= '1994-01-01'
+ and $l.l_receiptdate < '1995-01-01'
+ and ($l.l_shipmode = 'MAIL' or $l.l_shipmode = 'SHIP')
+group by $l_shipmode := $l.l_shipmode with $o
+order by $l_shipmode
+return {
+ "l_shipmode": $l_shipmode,
+ "high_line_count": sum(
+ for $i in $o
+ return
+ switch-case($i.o_orderpriority ='1-URGENT' or $i.o_orderpriority ='2-HIGH',
+ true, 1, false, 0)
+ ),
+ "low_line_count": sum(
+ for $i in $o
+ return switch-case($i.o_orderpriority ='1-URGENT' or $i.o_orderpriority ='2-HIGH',
+ true, 0, false, 1)
+ )
+}
+
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q13_customer_distribution.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q13_customer_distribution.3.query.aql
new file mode 100644
index 0000000..a7b7b62
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q13_customer_distribution.3.query.aql
@@ -0,0 +1,29 @@
+use dataverse tpch;
+set import-private-functions 'true';
+
+for $gco in (
+ for $co in (
+ for $c in dataset('Customer')
+ return {
+ "c_custkey": $c.c_custkey,
+ "o_orderkey_count": count(
+ for $o in dataset('Orders')
+ where $c.c_custkey = $o.o_custkey and not(like($o.o_comment,'%special%requests%'))
+ return $o.o_orderkey
+ )
+ }
+ )
+ group by $c_custkey := $co.c_custkey with $co
+ return {
+ "c_custkey": $c_custkey,
+ "c_count": sum(for $i in $co return $i.o_orderkey_count)
+ }
+)
+group by $c_count := $gco.c_count with $gco
+let $custdist := count($gco)
+order by $custdist desc, $c_count desc
+return {
+ "c_count": $c_count,
+ "custdist": $custdist
+}
+
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q14_promotion_effect.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q14_promotion_effect.3.query.aql
new file mode 100644
index 0000000..e6007e7
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q14_promotion_effect.3.query.aql
@@ -0,0 +1,20 @@
+use dataverse tpch;
+
+for $l in dataset('LineItem')
+for $p in dataset('Part')
+where $l.l_partkey = $p.p_partkey
+ and $l.l_shipdate >= '1995-09-01'
+ and $l.l_shipdate < '1995-10-01'
+let $lp := {
+ "l_extendedprice": $l.l_extendedprice,
+ "l_discount": $l.l_discount,
+ "p_type": $p.p_type
+ }
+group by $t:=1 with $lp
+return 100.00 * sum(
+ for $i in $lp
+ return switch-case(like($i.p_type, 'PROMO%'),
+ true, $i.l_extendedprice*(1-$i.l_discount),
+ false, 0.0)
+ ) / sum(for $i in $lp return $i.l_extendedprice * (1 - $i.l_discount)
+)
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q15_top_supplier.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q15_top_supplier.3.query.aql
new file mode 100644
index 0000000..45b0fad
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q15_top_supplier.3.query.aql
@@ -0,0 +1,27 @@
+use dataverse tpch;
+
+declare function revenue() {
+ for $l in dataset('LineItem')
+ where $l.l_shipdate >= '1996-01-01' and $l.l_shipdate < '1996-04-01'
+ group by $l_suppkey := $l.l_suppkey with $l
+ return {
+ "supplier_no": $l_suppkey,
+ "total_revenue": sum(for $i in $l return $i.l_extendedprice * (1 - $i.l_discount))
+ }
+}
+
+let $m := max(
+ for $r2 in revenue()
+ return $r2.total_revenue
+)
+
+for $s in dataset('Supplier')
+for $r in revenue()
+where $s.s_suppkey = $r.supplier_no and $r.total_revenue<$m+0.000000001 and $r.total_revenue>$m-0.000000001
+return {
+ "s_suppkey": $s.s_suppkey,
+ "s_name": $s.s_name,
+ "s_address": $s.s_address,
+ "s_phone": $s.s_phone,
+ "total_revenue": $r.total_revenue
+}
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q16_parts_supplier_relationship.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q16_parts_supplier_relationship.3.query.aql
new file mode 100644
index 0000000..014ceff
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q16_parts_supplier_relationship.3.query.aql
@@ -0,0 +1,48 @@
+use dataverse tpch;
+
+declare function tmp(){
+ for $psp in (
+ for $ps in dataset('Partsupp')
+ for $p in dataset('Part')
+ where $p.p_partkey = $ps.ps_partkey and $p.p_brand != 'Brand#45'
+ and not(like($p.p_type, 'MEDIUM POLISHED%'))
+ return {
+ "p_brand": $p.p_brand,
+ "p_type": $p.p_type,
+ "p_size": $p.p_size,
+ "ps_suppkey": $ps.ps_suppkey
+ }
+ )
+ for $s in dataset('Supplier')
+ where $psp.ps_suppkey = $s.s_suppkey and not(like($s.s_comment, '%Customer%Complaints%'))
+ return {
+ "p_brand": $psp.p_brand,
+ "p_type": $psp.p_type,
+ "p_size": $psp.p_size,
+ "ps_suppkey": $psp.ps_suppkey
+ }
+}
+
+for $t2 in (
+ for $t in tmp()
+ where $t.p_size = 49 or $t.p_size = 14 or $t.p_size = 23
+ or $t.p_size = 45 or $t.p_size = 19 or $t.p_size = 3
+ or $t.p_size = 36 or $t.p_size = 9
+ group by $p_brand1:= $t.p_brand, $p_type1 := $t.p_type,
+ $p_size1:= $t.p_size, $ps_suppkey1:=$t.ps_suppkey with $t
+ return {
+ "p_brand": $p_brand1,
+ "p_type": $p_type1,
+ "p_size": $p_size1,
+ "ps_suppkey": $ps_suppkey1
+ }
+)
+group by $p_brand := $t2.p_brand, $p_type := $t2.p_type, $p_size := $t2.p_size with $t2
+let $supplier_cnt := count(for $i in $t2 return $i.ps_suppkey)
+order by $supplier_cnt desc, $p_brand, $p_type, $p_size
+return {
+ "p_brand": $p_brand,
+ "p_type": $p_type,
+ "p_size": $p_size,
+ "supplier_cnt": $supplier_cnt
+}
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q17_small_quantity_order_revenue.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q17_small_quantity_order_revenue.3.query.aql
new file mode 100644
index 0000000..b92e623
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q17_small_quantity_order_revenue.3.query.aql
@@ -0,0 +1,23 @@
+use dataverse tpch;
+
+declare function tmp(){
+ for $l in dataset('LineItem')
+ group by $l_partkey := $l.l_partkey with $l
+ return {
+ "t_partkey": $l_partkey,
+ "t_avg_quantity": 0.2 * avg(for $i in $l return $i.l_quantity)
+ }
+}
+
+sum(
+ for $l in dataset('LineItem')
+ for $p in dataset('Part')
+ where $p.p_partkey = $l.l_partkey
+ and $p.p_brand = 'Brand#23'
+ and $p.p_container = 'MED BOX'
+ for $t in tmp()
+ where $l.l_partkey = $t.t_partkey
+ and $l.l_quantity < $t.t_avg_quantity
+ return $l.l_extendedprice
+)/7.0
+
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q18_large_volume_customer.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q18_large_volume_customer.3.query.aql
new file mode 100644
index 0000000..86b95cf
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q18_large_volume_customer.3.query.aql
@@ -0,0 +1,29 @@
+use dataverse tpch;
+
+for $c in dataset('Customer')
+for $o in dataset('Orders')
+where $c.c_custkey = $o.o_custkey
+for $t in (
+ for $l in dataset('LineItem')
+ group by $l_orderkey := $l.l_orderkey with $l
+ return {
+ "l_orderkey": $l_orderkey,
+ "t_sum_quantity": sum(for $i in $l return $i.l_quantity)
+ }
+)
+where $o.o_orderkey = $t.l_orderkey and $t.t_sum_quantity > 300
+for $l in dataset('LineItem')
+where $l.l_orderkey = $o.o_orderkey
+group by $c_name := $c.c_name, $c_custkey := $c.c_custkey, $o_orderkey := $o.o_orderkey,
+ $o_orderdate := $o.o_orderdate, $o_totalprice := $o.o_totalprice with $l
+order by $o_totalprice desc, $o_orderdate
+limit 100
+return {
+ "c_name": $c_name,
+ "c_custkey": $c_custkey,
+ "o_orderkey": $o_orderkey,
+ "o_orderdate": $o_orderdate,
+ "o_totalprice": $o_totalprice,
+ "sum_quantity": sum(for $j in $l return $j.l_quantity)
+}
+
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q19_discounted_revenue.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q19_discounted_revenue.3.query.aql
new file mode 100644
index 0000000..9ef0088
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q19_discounted_revenue.3.query.aql
@@ -0,0 +1,33 @@
+use dataverse tpch;
+
+set import-private-functions 'true';
+
+sum(
+ for $l in dataset('LineItem')
+ for $p in dataset('Part')
+ where $p.p_partkey = $l.l_partkey
+ and ( (
+ $p.p_brand = 'Brand#12'
+ and reg-exp($p.p_container,'SM CASE|SM BOX|SM PACK|SM PKG')
+ and $l.l_quantity >= 1 and $l.l_quantity <= 11
+ and $p.p_size >= 1 and $p.p_size <= 5
+ and ($l.l_shipmode = 'AIR' or $l.l_shipmode = 'AIR REG')
+ and $l.l_shipinstruct = 'DELIVER IN PERSON'
+ ) or (
+ $p.p_brand = 'Brand#23'
+ and reg-exp($p.p_container, 'MED BAG|MED BOX|MED PKG|MED PACK')
+ and $l.l_quantity >= 10 and $l.l_quantity <= 20
+ and $p.p_size >= 1 and $p.p_size <= 10
+ and ($l.l_shipmode = 'AIR' or $l.l_shipmode = 'AIR REG')
+ and $l.l_shipinstruct = 'DELIVER IN PERSON'
+ ) or (
+ $p.p_brand = 'Brand#34'
+ and reg-exp($p.p_container, 'LG CASE|LG BOX|LG PACK|LG PKG')
+ and $l.l_quantity >= 20 and $l.l_quantity <= 30
+ and $p.p_size >= 1 and $p.p_size <= 15
+ and ($l.l_shipmode = 'AIR' or $l.l_shipmode = 'AIR REG')
+ and $l.l_shipinstruct = 'DELIVER IN PERSON'
+ )
+ )
+ return $l.l_extendedprice * (1 - $l.l_discount)
+)
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q20_potential_part_promotion.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q20_potential_part_promotion.3.query.aql
new file mode 100644
index 0000000..096d558
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q20_potential_part_promotion.3.query.aql
@@ -0,0 +1,50 @@
+use dataverse tpch;
+
+for $t3 in (
+ for $t2 in (
+ for $l in dataset('LineItem')
+ where $l.l_shipdate >= '1994-01-01' and $l.l_shipdate < '1995-01-01'
+ group by $l_partkey:=$l.l_partkey, $l_suppkey:=$l.l_suppkey with $l
+ return {
+ "l_partkey": $l_partkey,
+ "l_suppkey": $l_suppkey,
+ "sum_quantity": 0.5 * sum(for $i in $l return $i.l_quantity)
+ }
+ )
+ for $pst1 in (
+ for $ps in dataset('Partsupp')
+ for $t1 in (
+ for $p in dataset('Part')
+ where like($p.p_name, 'forest%')
+ distinct by $p.p_partkey
+ return { "p_partkey": $p.p_partkey }
+ )
+ where $ps.ps_partkey = $t1.p_partkey
+ return {
+ "ps_suppkey": $ps.ps_suppkey,
+ "ps_partkey": $ps.ps_partkey,
+ "ps_availqty": $ps.ps_availqty
+ }
+ )
+ where $pst1.ps_partkey = $t2.l_partkey and $pst1.ps_suppkey = $t2.l_suppkey
+ and $pst1.ps_availqty > $t2.sum_quantity
+ distinct by $pst1.ps_suppkey
+ return { "ps_suppkey": $pst1.ps_suppkey }
+)
+for $t4 in (
+ for $n in dataset('Nation')
+ for $s in dataset('Supplier')
+ where $s.s_nationkey = $n.n_nationkey and $n.n_name = 'CANADA'
+ return {
+ "s_name": $s.s_name,
+ "s_address": $s.s_address,
+ "s_suppkey": $s.s_suppkey
+ }
+)
+where $t3.ps_suppkey = $t4.s_suppkey
+order by $t4.s_name
+return {
+ "s_name": $t4.s_name,
+ "s_address": $t4.s_address
+}
+
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q21_suppliers_who_kept_orders_waiting.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q21_suppliers_who_kept_orders_waiting.3.query.aql
new file mode 100644
index 0000000..b4daae1
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q21_suppliers_who_kept_orders_waiting.3.query.aql
@@ -0,0 +1,96 @@
+use dataverse tpch;
+
+declare function tmp1() {
+ for $l2 in (
+ for $l in dataset('LineItem')
+ group by $l_orderkey1 := $l.l_orderkey, $l_suppkey1 := $l.l_suppkey with $l
+ return {
+ "l_orderkey": $l_orderkey1,
+ "l_suppkey": $l_suppkey1
+ }
+ )
+ group by $l_orderkey := $l2.l_orderkey with $l2
+ return {
+ "l_orderkey": $l_orderkey,
+ "count_suppkey": count(for $i in $l2 return $i.l_suppkey),
+ "max_suppkey": max(for $i in $l2 return $i.l_suppkey)
+ }
+}
+
+declare function tmp2() {
+ for $l2 in (
+ for $l in dataset('LineItem')
+ group by $l_orderkey1 := $l.l_orderkey, $l_suppkey1 := $l.l_suppkey with $l
+ return {
+ "l_orderkey": $l_orderkey1,
+ "l_suppkey": $l_suppkey1,
+ "count": count(for $i in $l return $i.l_suppkey)
+ }
+ )
+ for $l3 in (
+ for $l in dataset('LineItem')
+ where $l.l_receiptdate <= $l.l_commitdate
+ group by $l_orderkey1 := $l.l_orderkey, $l_suppkey1 := $l.l_suppkey with $l
+ return {
+ "l_orderkey": $l_orderkey1,
+ "l_suppkey": $l_suppkey1,
+ "count": count(for $i in $l return $i.l_suppkey)
+ }
+ )
+ where $l2.l_orderkey = $l3.l_orderkey
+ and $l2.l_suppkey = $l3.l_suppkey
+ and $l2.count = $l3.count
+ group by $l_orderkey := $l2.l_orderkey with $l2
+ return {
+ "l_orderkey": $l_orderkey,
+ "count_suppkey": count(for $i in $l2 return $i.l_suppkey),
+ "max_suppkey": max(for $i in $l2 return $i.l_suppkey)
+ }
+}
+
+for $t4 in (
+ for $t3 in (
+ for $l in dataset('LineItem')
+ for $ns in (
+ for $n in dataset('Nation')
+ for $s in dataset('Supplier')
+ where $s.s_nationkey = $n.n_nationkey
+ and $n.n_name = 'SAUDI ARABIA'
+ return {
+ "s_name": $s.s_name,
+ "s_suppkey": $s.s_suppkey
+ }
+ )
+ where $ns.s_suppkey = $l.l_suppkey and $l.l_receiptdate > $l.l_commitdate
+ for $o in dataset('Orders')
+ where $o.o_orderkey = $l.l_orderkey
+ and $o.o_orderstatus = 'F'
+ for $t1 in tmp1()
+ where $l.l_orderkey = $t1.l_orderkey
+ and $t1.count_suppkey > 1
+ return {
+ "s_name": $ns.s_name,
+ "l_orderkey": $t1.l_orderkey,
+ "l_suppkey": $l.l_suppkey,
+ "t1_count_suppkey": $t1.count_suppkey
+ }
+ )
+ for $t2 in tmp2()
+ where $t3.l_orderkey = $t2.l_orderkey
+ and $t2.count_suppkey = $t3.t1_count_suppkey - 1
+ return {
+ "s_name": $t3.s_name,
+ "l_suppkey": $t3.l_suppkey,
+ "l_orderkey": $t2.l_orderkey,
+ "count_suppkey": $t2.count_suppkey,
+ "max_suppkey": $t2.max_suppkey
+ }
+)
+group by $s_name := $t4.s_name with $t4
+let $numwait := count($t4)
+order by $numwait desc, $s_name
+limit 100
+return {
+ "s_name": $s_name,
+ "numwait": $numwait
+}
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q21_suppliers_who_kept_orders_waiting.3.query_2.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q21_suppliers_who_kept_orders_waiting.3.query_2.aql
new file mode 100644
index 0000000..3e5f420
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q21_suppliers_who_kept_orders_waiting.3.query_2.aql
@@ -0,0 +1,31 @@
+use dataverse tpch;
+
+for $x in (
+ for $n in dataset Nation
+ where $n.n_name = 'SAUDI ARABIA'
+ for $s in dataset Supplier
+ where $s.s_nationkey = $n.n_nationkey
+ for $l1 in dataset LineItem
+ where $s.s_suppkey = $l1.l_suppkey
+ and $l1.l_receiptdate > $l1.l_commitdate
+ for $o in dataset Orders
+ where $o.o_orderstatus = 'F'
+ and $o.o_orderkey = $l1.l_orderkey
+ and (some $l2 in dataset LineItem
+ satisfies $l1.l_orderkey = $l2.l_orderkey
+ and $l1.l_suppkey != $l2.l_suppkey)
+ and (every $l3 in dataset LineItem
+ satisfies $l1.l_orderkey != $l3.l_orderkey
+ or $l1.l_suppkey = $l3.l_suppkey
+ or $l3.l_receiptdate <= $l3.l_commitdate)
+ return {
+ "s_name" : $s.s_name
+ }
+)
+group by $s_name := $x.s_name with $x
+let $numwait := count($x)
+order by $numwait desc
+return {
+ "s_name": $s_name,
+ "numwait": $numwait
+}
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/queries/q22_global_sales_opportunity.3.query.aql b/asterix-benchmarks/src/main/resources/tpc-h/queries/q22_global_sales_opportunity.3.query.aql
new file mode 100644
index 0000000..5c7d95d
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/queries/q22_global_sales_opportunity.3.query.aql
@@ -0,0 +1,33 @@
+use dataverse tpch;
+
+declare function q22_customer_tmp() {
+ for $c in dataset('Customer')
+ let $phone_substr := substring($c.c_phone, 1, 2)
+ where $phone_substr = '13'
+ or $phone_substr = '31'
+ or $phone_substr = '23'
+ or $phone_substr = '29'
+ or $phone_substr = '30'
+ or $phone_substr = '18'
+ or $phone_substr = '17'
+ return {
+ "c_acctbal": $c.c_acctbal,
+ "c_custkey": $c.c_custkey,
+ "cntrycode": $phone_substr
+ }
+}
+
+let $avg := avg(
+ for $c in q22_customer_tmp()
+ where $c.c_acctbal > 0.00
+ return $c.c_acctbal
+)
+for $ct in q22_customer_tmp()
+where $ct.c_acctbal > $avg
+group by $cntrycode := $ct.cntrycode with $ct
+order by $cntrycode
+return {
+ "cntrycode": $cntrycode,
+ "numcust": count($ct),
+ "totacctbal": sum(for $i in $ct return $i.c_acctbal)
+}
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q01_pricing_summary_report_nt.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q01_pricing_summary_report_nt.3.query.adm
new file mode 100644
index 0000000..ddb4b48
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q01_pricing_summary_report_nt.3.query.adm
@@ -0,0 +1,4 @@
+{ "l_returnflag": "A", "l_linestatus": "F", "sum_qty": 37734107, "sum_base_price": 5.658655440072989E10d, "sum_disc_price": 5.3758257134870056E10d, "sum_charge": 5.5909065222827835E10d, "ave_qty": 25.522005853257337d, "ave_price": 38273.129734621594d, "ave_disc": 0.04998529583842826d, "count_order": 1478493i64 }
+{ "l_returnflag": "N", "l_linestatus": "F", "sum_qty": 991417, "sum_base_price": 1.4875047103800046E9d, "sum_disc_price": 1.4130821680540953E9d, "sum_charge": 1.469649223194375E9d, "ave_qty": 25.516471920522985d, "ave_price": 38284.46776084842d, "ave_disc": 0.050093426674214014d, "count_order": 38854i64 }
+{ "l_returnflag": "N", "l_linestatus": "O", "sum_qty": 74476040, "sum_base_price": 1.1170172969774083E11d, "sum_disc_price": 1.0611823030760648E11d, "sum_charge": 1.1036704387249731E11d, "ave_qty": 25.50222676958499d, "ave_price": 38249.117988908554d, "ave_disc": 0.049996586053579505d, "count_order": 2920374i64 }
+{ "l_returnflag": "R", "l_linestatus": "F", "sum_qty": 37719753, "sum_base_price": 5.656804138089871E10d, "sum_disc_price": 5.374129268460493E10d, "sum_charge": 5.588961911983354E10d, "ave_qty": 25.50579361269077d, "ave_price": 38250.854626098786d, "ave_disc": 0.0500094058301571d, "count_order": 1478870i64 }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q02_minimum_cost_supplier.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q02_minimum_cost_supplier.3.query.adm
new file mode 100644
index 0000000..6fc3877
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q02_minimum_cost_supplier.3.query.adm
@@ -0,0 +1,100 @@
+{ "s_acctbal": 9938.53d, "s_name": "Supplier#000005359", "n_name": "UNITED KINGDOM", "p_partkey": 185358, "p_mfgr": "Manufacturer#4", "s_address": "QKuHYh,vZGiwu2FWEJoLDx04", "s_phone": "33-429-790-6131", "s_comment": "uriously regular requests hag" }
+{ "s_acctbal": 9937.84d, "s_name": "Supplier#000005969", "n_name": "ROMANIA", "p_partkey": 108438, "p_mfgr": "Manufacturer#1", "s_address": "ANDENSOSmk,miq23Xfb5RWt6dvUcvt6Qa", "s_phone": "29-520-692-3537", "s_comment": "efully express instructions. regular requests against the slyly fin" }
+{ "s_acctbal": 9936.22d, "s_name": "Supplier#000005250", "n_name": "UNITED KINGDOM", "p_partkey": 249, "p_mfgr": "Manufacturer#4", "s_address": "B3rqp0xbSEim4Mpy2RH J", "s_phone": "33-320-228-2957", "s_comment": "etect about the furiously final accounts. slyly ironic pinto beans sleep inside the furiously" }
+{ "s_acctbal": 9923.77d, "s_name": "Supplier#000002324", "n_name": "GERMANY", "p_partkey": 29821, "p_mfgr": "Manufacturer#4", "s_address": "y3OD9UywSTOk", "s_phone": "17-779-299-1839", "s_comment": "ackages boost blithely. blithely regular deposits c" }
+{ "s_acctbal": 9871.22d, "s_name": "Supplier#000006373", "n_name": "GERMANY", "p_partkey": 43868, "p_mfgr": "Manufacturer#5", "s_address": "J8fcXWsTqM", "s_phone": "17-813-485-8637", "s_comment": "etect blithely bold asymptotes. fluffily ironic platelets wake furiously; blit" }
+{ "s_acctbal": 9870.78d, "s_name": "Supplier#000001286", "n_name": "GERMANY", "p_partkey": 81285, "p_mfgr": "Manufacturer#2", "s_address": "YKA,E2fjiVd7eUrzp2Ef8j1QxGo2DFnosaTEH", "s_phone": "17-516-924-4574", "s_comment": " regular accounts. furiously unusual courts above the fi" }
+{ "s_acctbal": 9870.78d, "s_name": "Supplier#000001286", "n_name": "GERMANY", "p_partkey": 181285, "p_mfgr": "Manufacturer#4", "s_address": "YKA,E2fjiVd7eUrzp2Ef8j1QxGo2DFnosaTEH", "s_phone": "17-516-924-4574", "s_comment": " regular accounts. furiously unusual courts above the fi" }
+{ "s_acctbal": 9852.52d, "s_name": "Supplier#000008973", "n_name": "RUSSIA", "p_partkey": 18972, "p_mfgr": "Manufacturer#2", "s_address": "t5L67YdBYYH6o,Vz24jpDyQ9", "s_phone": "32-188-594-7038", "s_comment": "rns wake final foxes. carefully unusual depende" }
+{ "s_acctbal": 9847.83d, "s_name": "Supplier#000008097", "n_name": "RUSSIA", "p_partkey": 130557, "p_mfgr": "Manufacturer#2", "s_address": "xMe97bpE69NzdwLoX", "s_phone": "32-375-640-3593", "s_comment": " the special excuses. silent sentiments serve carefully final ac" }
+{ "s_acctbal": 9847.57d, "s_name": "Supplier#000006345", "n_name": "FRANCE", "p_partkey": 86344, "p_mfgr": "Manufacturer#1", "s_address": "VSt3rzk3qG698u6ld8HhOByvrTcSTSvQlDQDag", "s_phone": "16-886-766-7945", "s_comment": "ges. slyly regular requests are. ruthless, express excuses cajole blithely across the unu" }
+{ "s_acctbal": 9847.57d, "s_name": "Supplier#000006345", "n_name": "FRANCE", "p_partkey": 173827, "p_mfgr": "Manufacturer#2", "s_address": "VSt3rzk3qG698u6ld8HhOByvrTcSTSvQlDQDag", "s_phone": "16-886-766-7945", "s_comment": "ges. slyly regular requests are. ruthless, express excuses cajole blithely across the unu" }
+{ "s_acctbal": 9836.93d, "s_name": "Supplier#000007342", "n_name": "RUSSIA", "p_partkey": 4841, "p_mfgr": "Manufacturer#4", "s_address": "JOlK7C1,7xrEZSSOw", "s_phone": "32-399-414-5385", "s_comment": "blithely carefully bold theodolites. fur" }
+{ "s_acctbal": 9817.1d, "s_name": "Supplier#000002352", "n_name": "RUSSIA", "p_partkey": 124815, "p_mfgr": "Manufacturer#2", "s_address": "4LfoHUZjgjEbAKw TgdKcgOc4D4uCYw", "s_phone": "32-551-831-1437", "s_comment": "wake carefully alongside of the carefully final ex" }
+{ "s_acctbal": 9817.1d, "s_name": "Supplier#000002352", "n_name": "RUSSIA", "p_partkey": 152351, "p_mfgr": "Manufacturer#3", "s_address": "4LfoHUZjgjEbAKw TgdKcgOc4D4uCYw", "s_phone": "32-551-831-1437", "s_comment": "wake carefully alongside of the carefully final ex" }
+{ "s_acctbal": 9739.86d, "s_name": "Supplier#000003384", "n_name": "FRANCE", "p_partkey": 138357, "p_mfgr": "Manufacturer#2", "s_address": "o,Z3v4POifevE k9U1b 6J1ucX,I", "s_phone": "16-494-913-5925", "s_comment": "s after the furiously bold packages sleep fluffily idly final requests: quickly final" }
+{ "s_acctbal": 9721.95d, "s_name": "Supplier#000008757", "n_name": "UNITED KINGDOM", "p_partkey": 156241, "p_mfgr": "Manufacturer#3", "s_address": "Atg6GnM4dT2", "s_phone": "33-821-407-2995", "s_comment": "eep furiously sauternes; quickl" }
+{ "s_acctbal": 9681.33d, "s_name": "Supplier#000008406", "n_name": "RUSSIA", "p_partkey": 78405, "p_mfgr": "Manufacturer#1", "s_address": ",qUuXcftUl", "s_phone": "32-139-873-8571", "s_comment": "haggle slyly regular excuses. quic" }
+{ "s_acctbal": 9643.55d, "s_name": "Supplier#000005148", "n_name": "ROMANIA", "p_partkey": 107617, "p_mfgr": "Manufacturer#1", "s_address": "kT4ciVFslx9z4s79p Js825", "s_phone": "29-252-617-4850", "s_comment": "final excuses. final ideas boost quickly furiously speci" }
+{ "s_acctbal": 9624.82d, "s_name": "Supplier#000001816", "n_name": "FRANCE", "p_partkey": 34306, "p_mfgr": "Manufacturer#3", "s_address": "e7vab91vLJPWxxZnewmnDBpDmxYHrb", "s_phone": "16-392-237-6726", "s_comment": "e packages are around the special ideas. special, pending foxes us" }
+{ "s_acctbal": 9624.78d, "s_name": "Supplier#000009658", "n_name": "ROMANIA", "p_partkey": 189657, "p_mfgr": "Manufacturer#1", "s_address": "oE9uBgEfSS4opIcepXyAYM,x", "s_phone": "29-748-876-2014", "s_comment": "ronic asymptotes wake bravely final" }
+{ "s_acctbal": 9612.94d, "s_name": "Supplier#000003228", "n_name": "ROMANIA", "p_partkey": 120715, "p_mfgr": "Manufacturer#2", "s_address": "KDdpNKN3cWu7ZSrbdqp7AfSLxx,qWB", "s_phone": "29-325-784-8187", "s_comment": "warhorses. quickly even deposits sublate daringly ironic instructions. slyly blithe t" }
+{ "s_acctbal": 9612.94d, "s_name": "Supplier#000003228", "n_name": "ROMANIA", "p_partkey": 198189, "p_mfgr": "Manufacturer#4", "s_address": "KDdpNKN3cWu7ZSrbdqp7AfSLxx,qWB", "s_phone": "29-325-784-8187", "s_comment": "warhorses. quickly even deposits sublate daringly ironic instructions. slyly blithe t" }
+{ "s_acctbal": 9571.83d, "s_name": "Supplier#000004305", "n_name": "ROMANIA", "p_partkey": 179270, "p_mfgr": "Manufacturer#2", "s_address": "qNHZ7WmCzygwMPRDO9Ps", "s_phone": "29-973-481-1831", "s_comment": "kly carefully express asymptotes. furiou" }
+{ "s_acctbal": 9558.1d, "s_name": "Supplier#000003532", "n_name": "UNITED KINGDOM", "p_partkey": 88515, "p_mfgr": "Manufacturer#4", "s_address": "EOeuiiOn21OVpTlGguufFDFsbN1p0lhpxHp", "s_phone": "33-152-301-2164", "s_comment": " foxes. quickly even excuses use. slyly special foxes nag bl" }
+{ "s_acctbal": 9492.79d, "s_name": "Supplier#000005975", "n_name": "GERMANY", "p_partkey": 25974, "p_mfgr": "Manufacturer#5", "s_address": "S6mIiCTx82z7lV", "s_phone": "17-992-579-4839", "s_comment": "arefully pending accounts. blithely regular excuses boost carefully carefully ironic p" }
+{ "s_acctbal": 9461.05d, "s_name": "Supplier#000002536", "n_name": "UNITED KINGDOM", "p_partkey": 20033, "p_mfgr": "Manufacturer#1", "s_address": "8mmGbyzaU 7ZS2wJumTibypncu9pNkDc4FYA", "s_phone": "33-556-973-5522", "s_comment": ". slyly regular deposits wake slyly. furiously regular warthogs are." }
+{ "s_acctbal": 9453.01d, "s_name": "Supplier#000000802", "n_name": "ROMANIA", "p_partkey": 175767, "p_mfgr": "Manufacturer#1", "s_address": ",6HYXb4uaHITmtMBj4Ak57Pd", "s_phone": "29-342-882-6463", "s_comment": "gular frets. permanently special multipliers believe blithely alongs" }
+{ "s_acctbal": 9408.65d, "s_name": "Supplier#000007772", "n_name": "UNITED KINGDOM", "p_partkey": 117771, "p_mfgr": "Manufacturer#4", "s_address": "AiC5YAH,gdu0i7", "s_phone": "33-152-491-1126", "s_comment": "nag against the final requests. furiously unusual packages cajole blit" }
+{ "s_acctbal": 9359.61d, "s_name": "Supplier#000004856", "n_name": "ROMANIA", "p_partkey": 62349, "p_mfgr": "Manufacturer#5", "s_address": "HYogcF3Jb yh1", "s_phone": "29-334-870-9731", "s_comment": "y ironic theodolites. blithely sile" }
+{ "s_acctbal": 9357.45d, "s_name": "Supplier#000006188", "n_name": "UNITED KINGDOM", "p_partkey": 138648, "p_mfgr": "Manufacturer#1", "s_address": "g801,ssP8wpTk4Hm", "s_phone": "33-583-607-1633", "s_comment": "ously always regular packages. fluffily even accounts beneath the furiously final pack" }
+{ "s_acctbal": 9352.04d, "s_name": "Supplier#000003439", "n_name": "GERMANY", "p_partkey": 170921, "p_mfgr": "Manufacturer#4", "s_address": "qYPDgoiBGhCYxjgC", "s_phone": "17-128-996-4650", "s_comment": " according to the carefully bold ideas" }
+{ "s_acctbal": 9312.97d, "s_name": "Supplier#000007807", "n_name": "RUSSIA", "p_partkey": 90279, "p_mfgr": "Manufacturer#5", "s_address": "oGYMPCk9XHGB2PBfKRnHA", "s_phone": "32-673-872-5854", "s_comment": "ecial packages among the pending, even requests use regula" }
+{ "s_acctbal": 9312.97d, "s_name": "Supplier#000007807", "n_name": "RUSSIA", "p_partkey": 100276, "p_mfgr": "Manufacturer#5", "s_address": "oGYMPCk9XHGB2PBfKRnHA", "s_phone": "32-673-872-5854", "s_comment": "ecial packages among the pending, even requests use regula" }
+{ "s_acctbal": 9280.27d, "s_name": "Supplier#000007194", "n_name": "ROMANIA", "p_partkey": 47193, "p_mfgr": "Manufacturer#3", "s_address": "zhRUQkBSrFYxIAXTfInj vyGRQjeK", "s_phone": "29-318-454-2133", "s_comment": "o beans haggle after the furiously unusual deposits. carefully silent dolphins cajole carefully" }
+{ "s_acctbal": 9274.8d, "s_name": "Supplier#000008854", "n_name": "RUSSIA", "p_partkey": 76346, "p_mfgr": "Manufacturer#3", "s_address": "1xhLoOUM7I3mZ1mKnerw OSqdbb4QbGa", "s_phone": "32-524-148-5221", "s_comment": "y. courts do wake slyly. carefully ironic platelets haggle above the slyly regular the" }
+{ "s_acctbal": 9249.35d, "s_name": "Supplier#000003973", "n_name": "FRANCE", "p_partkey": 26466, "p_mfgr": "Manufacturer#1", "s_address": "d18GiDsL6Wm2IsGXM,RZf1jCsgZAOjNYVThTRP4", "s_phone": "16-722-866-1658", "s_comment": "uests are furiously. regular tithes through the regular, final accounts cajole furiously above the q" }
+{ "s_acctbal": 9249.35d, "s_name": "Supplier#000003973", "n_name": "FRANCE", "p_partkey": 33972, "p_mfgr": "Manufacturer#1", "s_address": "d18GiDsL6Wm2IsGXM,RZf1jCsgZAOjNYVThTRP4", "s_phone": "16-722-866-1658", "s_comment": "uests are furiously. regular tithes through the regular, final accounts cajole furiously above the q" }
+{ "s_acctbal": 9208.7d, "s_name": "Supplier#000007769", "n_name": "ROMANIA", "p_partkey": 40256, "p_mfgr": "Manufacturer#5", "s_address": "rsimdze 5o9P Ht7xS", "s_phone": "29-964-424-9649", "s_comment": "lites was quickly above the furiously ironic requests. slyly even foxes against the blithely bold " }
+{ "s_acctbal": 9201.47d, "s_name": "Supplier#000009690", "n_name": "UNITED KINGDOM", "p_partkey": 67183, "p_mfgr": "Manufacturer#5", "s_address": "CB BnUTlmi5zdeEl7R7", "s_phone": "33-121-267-9529", "s_comment": "e even, even foxes. blithely ironic packages cajole regular packages. slyly final ide" }
+{ "s_acctbal": 9192.1d, "s_name": "Supplier#000000115", "n_name": "UNITED KINGDOM", "p_partkey": 85098, "p_mfgr": "Manufacturer#3", "s_address": "nJ 2t0f7Ve,wL1,6WzGBJLNBUCKlsV", "s_phone": "33-597-248-1220", "s_comment": "es across the carefully express accounts boost caref" }
+{ "s_acctbal": 9189.98d, "s_name": "Supplier#000001226", "n_name": "GERMANY", "p_partkey": 21225, "p_mfgr": "Manufacturer#4", "s_address": "qsLCqSvLyZfuXIpjz", "s_phone": "17-725-903-1381", "s_comment": " deposits. blithely bold excuses about the slyly bold forges wake " }
+{ "s_acctbal": 9128.97d, "s_name": "Supplier#000004311", "n_name": "RUSSIA", "p_partkey": 146768, "p_mfgr": "Manufacturer#5", "s_address": "I8IjnXd7NSJRs594RxsRR0", "s_phone": "32-155-440-7120", "s_comment": "refully. blithely unusual asymptotes haggle " }
+{ "s_acctbal": 9104.83d, "s_name": "Supplier#000008520", "n_name": "GERMANY", "p_partkey": 150974, "p_mfgr": "Manufacturer#4", "s_address": "RqRVDgD0ER J9 b41vR2,3", "s_phone": "17-728-804-1793", "s_comment": "ly about the blithely ironic depths. slyly final theodolites among the fluffily bold ideas print" }
+{ "s_acctbal": 9101.0d, "s_name": "Supplier#000005791", "n_name": "ROMANIA", "p_partkey": 128254, "p_mfgr": "Manufacturer#5", "s_address": "zub2zCV,jhHPPQqi,P2INAjE1zI n66cOEoXFG", "s_phone": "29-549-251-5384", "s_comment": "ts. notornis detect blithely above the carefully bold requests. blithely even package" }
+{ "s_acctbal": 9094.57d, "s_name": "Supplier#000004582", "n_name": "RUSSIA", "p_partkey": 39575, "p_mfgr": "Manufacturer#1", "s_address": "WB0XkCSG3r,mnQ n,h9VIxjjr9ARHFvKgMDf", "s_phone": "32-587-577-1351", "s_comment": "jole. regular accounts sleep blithely frets. final pinto beans play furiously past the " }
+{ "s_acctbal": 8996.87d, "s_name": "Supplier#000004702", "n_name": "FRANCE", "p_partkey": 102191, "p_mfgr": "Manufacturer#5", "s_address": "8XVcQK23akp", "s_phone": "16-811-269-8946", "s_comment": "ickly final packages along the express plat" }
+{ "s_acctbal": 8996.14d, "s_name": "Supplier#000009814", "n_name": "ROMANIA", "p_partkey": 139813, "p_mfgr": "Manufacturer#2", "s_address": "af0O5pg83lPU4IDVmEylXZVqYZQzSDlYLAmR", "s_phone": "29-995-571-8781", "s_comment": " dependencies boost quickly across the furiously pending requests! unusual dolphins play sl" }
+{ "s_acctbal": 8968.42d, "s_name": "Supplier#000010000", "n_name": "ROMANIA", "p_partkey": 119999, "p_mfgr": "Manufacturer#5", "s_address": "aTGLEusCiL4F PDBdv665XBJhPyCOB0i", "s_phone": "29-578-432-2146", "s_comment": "ly regular foxes boost slyly. quickly special waters boost carefully ironi" }
+{ "s_acctbal": 8936.82d, "s_name": "Supplier#000007043", "n_name": "UNITED KINGDOM", "p_partkey": 109512, "p_mfgr": "Manufacturer#1", "s_address": "FVajceZInZdbJE6Z9XsRUxrUEpiwHDrOXi,1Rz", "s_phone": "33-784-177-8208", "s_comment": "efully regular courts. furiousl" }
+{ "s_acctbal": 8929.42d, "s_name": "Supplier#000008770", "n_name": "FRANCE", "p_partkey": 173735, "p_mfgr": "Manufacturer#4", "s_address": "R7cG26TtXrHAP9 HckhfRi", "s_phone": "16-242-746-9248", "s_comment": "cajole furiously unusual requests. quickly stealthy requests are. " }
+{ "s_acctbal": 8920.59d, "s_name": "Supplier#000003967", "n_name": "ROMANIA", "p_partkey": 26460, "p_mfgr": "Manufacturer#1", "s_address": "eHoAXe62SY9", "s_phone": "29-194-731-3944", "s_comment": "aters. express, pending instructions sleep. brave, r" }
+{ "s_acctbal": 8920.59d, "s_name": "Supplier#000003967", "n_name": "ROMANIA", "p_partkey": 173966, "p_mfgr": "Manufacturer#2", "s_address": "eHoAXe62SY9", "s_phone": "29-194-731-3944", "s_comment": "aters. express, pending instructions sleep. brave, r" }
+{ "s_acctbal": 8913.96d, "s_name": "Supplier#000004603", "n_name": "UNITED KINGDOM", "p_partkey": 137063, "p_mfgr": "Manufacturer#2", "s_address": "OUzlvMUr7n,utLxmPNeYKSf3T24OXskxB5", "s_phone": "33-789-255-7342", "s_comment": " haggle slyly above the furiously regular pinto beans. even " }
+{ "s_acctbal": 8877.82d, "s_name": "Supplier#000007967", "n_name": "FRANCE", "p_partkey": 167966, "p_mfgr": "Manufacturer#5", "s_address": "A3pi1BARM4nx6R,qrwFoRPU", "s_phone": "16-442-147-9345", "s_comment": "ously foxes. express, ironic requests im" }
+{ "s_acctbal": 8862.24d, "s_name": "Supplier#000003323", "n_name": "ROMANIA", "p_partkey": 73322, "p_mfgr": "Manufacturer#3", "s_address": "W9 lYcsC9FwBqk3ItL", "s_phone": "29-736-951-3710", "s_comment": "ly pending ideas sleep about the furiously unu" }
+{ "s_acctbal": 8841.59d, "s_name": "Supplier#000005750", "n_name": "ROMANIA", "p_partkey": 100729, "p_mfgr": "Manufacturer#5", "s_address": "Erx3lAgu0g62iaHF9x50uMH4EgeN9hEG", "s_phone": "29-344-502-5481", "s_comment": "gainst the pinto beans. fluffily unusual dependencies affix slyly even deposits." }
+{ "s_acctbal": 8781.71d, "s_name": "Supplier#000003121", "n_name": "ROMANIA", "p_partkey": 13120, "p_mfgr": "Manufacturer#5", "s_address": "wNqTogx238ZYCamFb,50v,bj 4IbNFW9Bvw1xP", "s_phone": "29-707-291-5144", "s_comment": "s wake quickly ironic ideas" }
+{ "s_acctbal": 8754.24d, "s_name": "Supplier#000009407", "n_name": "UNITED KINGDOM", "p_partkey": 179406, "p_mfgr": "Manufacturer#4", "s_address": "CHRCbkaWcf5B", "s_phone": "33-903-970-9604", "s_comment": "e ironic requests. carefully even foxes above the furious" }
+{ "s_acctbal": 8691.06d, "s_name": "Supplier#000004429", "n_name": "UNITED KINGDOM", "p_partkey": 126892, "p_mfgr": "Manufacturer#2", "s_address": "k,BQms5UhoAF1B2Asi,fLib", "s_phone": "33-964-337-5038", "s_comment": "efully express deposits kindle after the deposits. final " }
+{ "s_acctbal": 8655.99d, "s_name": "Supplier#000006330", "n_name": "RUSSIA", "p_partkey": 193810, "p_mfgr": "Manufacturer#2", "s_address": "UozlaENr0ytKe2w6CeIEWFWn iO3S8Rae7Ou", "s_phone": "32-561-198-3705", "s_comment": "symptotes use about the express dolphins. requests use after the express platelets. final, ex" }
+{ "s_acctbal": 8638.36d, "s_name": "Supplier#000002920", "n_name": "RUSSIA", "p_partkey": 75398, "p_mfgr": "Manufacturer#1", "s_address": "Je2a8bszf3L", "s_phone": "32-122-621-7549", "s_comment": "ly quickly ironic requests. even requests whithout t" }
+{ "s_acctbal": 8638.36d, "s_name": "Supplier#000002920", "n_name": "RUSSIA", "p_partkey": 170402, "p_mfgr": "Manufacturer#3", "s_address": "Je2a8bszf3L", "s_phone": "32-122-621-7549", "s_comment": "ly quickly ironic requests. even requests whithout t" }
+{ "s_acctbal": 8607.69d, "s_name": "Supplier#000006003", "n_name": "UNITED KINGDOM", "p_partkey": 76002, "p_mfgr": "Manufacturer#2", "s_address": "EH9wADcEiuenM0NR08zDwMidw,52Y2RyILEiA", "s_phone": "33-416-807-5206", "s_comment": "ar, pending accounts. pending depende" }
+{ "s_acctbal": 8569.52d, "s_name": "Supplier#000005936", "n_name": "RUSSIA", "p_partkey": 5935, "p_mfgr": "Manufacturer#5", "s_address": "jXaNZ6vwnEWJ2ksLZJpjtgt0bY2a3AU", "s_phone": "32-644-251-7916", "s_comment": ". regular foxes nag carefully atop the regular, silent deposits. quickly regular packages " }
+{ "s_acctbal": 8564.12d, "s_name": "Supplier#000000033", "n_name": "GERMANY", "p_partkey": 110032, "p_mfgr": "Manufacturer#1", "s_address": "gfeKpYw3400L0SDywXA6Ya1Qmq1w6YB9f3R", "s_phone": "17-138-897-9374", "s_comment": "n sauternes along the regular asymptotes are regularly along the " }
+{ "s_acctbal": 8553.82d, "s_name": "Supplier#000003979", "n_name": "ROMANIA", "p_partkey": 143978, "p_mfgr": "Manufacturer#4", "s_address": "BfmVhCAnCMY3jzpjUMy4CNWs9 HzpdQR7INJU", "s_phone": "29-124-646-4897", "s_comment": "ic requests wake against the blithely unusual accounts. fluffily r" }
+{ "s_acctbal": 8517.23d, "s_name": "Supplier#000009529", "n_name": "RUSSIA", "p_partkey": 37025, "p_mfgr": "Manufacturer#5", "s_address": "e44R8o7JAIS9iMcr", "s_phone": "32-565-297-8775", "s_comment": "ove the even courts. furiously special platelets " }
+{ "s_acctbal": 8517.23d, "s_name": "Supplier#000009529", "n_name": "RUSSIA", "p_partkey": 59528, "p_mfgr": "Manufacturer#2", "s_address": "e44R8o7JAIS9iMcr", "s_phone": "32-565-297-8775", "s_comment": "ove the even courts. furiously special platelets " }
+{ "s_acctbal": 8503.7d, "s_name": "Supplier#000006830", "n_name": "RUSSIA", "p_partkey": 44325, "p_mfgr": "Manufacturer#4", "s_address": "BC4WFCYRUZyaIgchU 4S", "s_phone": "32-147-878-5069", "s_comment": "pades cajole. furious packages among the carefully express excuses boost furiously across th" }
+{ "s_acctbal": 8457.09d, "s_name": "Supplier#000009456", "n_name": "UNITED KINGDOM", "p_partkey": 19455, "p_mfgr": "Manufacturer#1", "s_address": "7SBhZs8gP1cJjT0Qf433YBk", "s_phone": "33-858-440-4349", "s_comment": "cing requests along the furiously unusual deposits promise among the furiously unus" }
+{ "s_acctbal": 8441.4d, "s_name": "Supplier#000003817", "n_name": "FRANCE", "p_partkey": 141302, "p_mfgr": "Manufacturer#2", "s_address": "hU3fz3xL78", "s_phone": "16-339-356-5115", "s_comment": "ely even ideas. ideas wake slyly furiously unusual instructions. pinto beans sleep ag" }
+{ "s_acctbal": 8432.89d, "s_name": "Supplier#000003990", "n_name": "RUSSIA", "p_partkey": 191470, "p_mfgr": "Manufacturer#1", "s_address": "wehBBp1RQbfxAYDASS75MsywmsKHRVdkrvNe6m", "s_phone": "32-839-509-9301", "s_comment": "ep furiously. packages should have to haggle slyly across the deposits. furiously regu" }
+{ "s_acctbal": 8431.4d, "s_name": "Supplier#000002675", "n_name": "ROMANIA", "p_partkey": 5174, "p_mfgr": "Manufacturer#1", "s_address": "HJFStOu9R5NGPOegKhgbzBdyvrG2yh8w", "s_phone": "29-474-643-1443", "s_comment": "ithely express pinto beans. blithely even foxes haggle. furiously regular theodol" }
+{ "s_acctbal": 8407.04d, "s_name": "Supplier#000005406", "n_name": "RUSSIA", "p_partkey": 162889, "p_mfgr": "Manufacturer#4", "s_address": "j7 gYF5RW8DC5UrjKC", "s_phone": "32-626-152-4621", "s_comment": "r the blithely regular packages. slyly ironic theodoli" }
+{ "s_acctbal": 8386.08d, "s_name": "Supplier#000008518", "n_name": "FRANCE", "p_partkey": 36014, "p_mfgr": "Manufacturer#3", "s_address": "2jqzqqAVe9crMVGP,n9nTsQXulNLTUYoJjEDcqWV", "s_phone": "16-618-780-7481", "s_comment": "blithely bold pains are carefully platelets. finally regular pinto beans sleep carefully special" }
+{ "s_acctbal": 8376.52d, "s_name": "Supplier#000005306", "n_name": "UNITED KINGDOM", "p_partkey": 190267, "p_mfgr": "Manufacturer#5", "s_address": "9t8Y8 QqSIsoADPt6NLdk,TP5zyRx41oBUlgoGc9", "s_phone": "33-632-514-7931", "s_comment": "ly final accounts sleep special, regular requests. furiously regular" }
+{ "s_acctbal": 8348.74d, "s_name": "Supplier#000008851", "n_name": "FRANCE", "p_partkey": 66344, "p_mfgr": "Manufacturer#4", "s_address": "nWxi7GwEbjhw1", "s_phone": "16-796-240-2472", "s_comment": " boldly final deposits. regular, even instructions detect slyly. fluffily unusual pinto bea" }
+{ "s_acctbal": 8338.58d, "s_name": "Supplier#000007269", "n_name": "FRANCE", "p_partkey": 17268, "p_mfgr": "Manufacturer#4", "s_address": "ZwhJSwABUoiB04,3", "s_phone": "16-267-277-4365", "s_comment": "iously final accounts. even pinto beans cajole slyly regular" }
+{ "s_acctbal": 8328.46d, "s_name": "Supplier#000001744", "n_name": "ROMANIA", "p_partkey": 69237, "p_mfgr": "Manufacturer#5", "s_address": "oLo3fV64q2,FKHa3p,qHnS7Yzv,ps8", "s_phone": "29-330-728-5873", "s_comment": "ep carefully-- even, careful packages are slyly along t" }
+{ "s_acctbal": 8307.93d, "s_name": "Supplier#000003142", "n_name": "GERMANY", "p_partkey": 18139, "p_mfgr": "Manufacturer#1", "s_address": "dqblvV8dCNAorGlJ", "s_phone": "17-595-447-6026", "s_comment": "olites wake furiously regular decoys. final requests nod " }
+{ "s_acctbal": 8231.61d, "s_name": "Supplier#000009558", "n_name": "RUSSIA", "p_partkey": 192000, "p_mfgr": "Manufacturer#2", "s_address": "mcdgen,yT1iJDHDS5fV", "s_phone": "32-762-137-5858", "s_comment": " foxes according to the furi" }
+{ "s_acctbal": 8152.61d, "s_name": "Supplier#000002731", "n_name": "ROMANIA", "p_partkey": 15227, "p_mfgr": "Manufacturer#4", "s_address": " nluXJCuY1tu", "s_phone": "29-805-463-2030", "s_comment": " special requests. even, regular warhorses affix among the final gr" }
+{ "s_acctbal": 8109.09d, "s_name": "Supplier#000009186", "n_name": "FRANCE", "p_partkey": 99185, "p_mfgr": "Manufacturer#1", "s_address": "wgfosrVPexl9pEXWywaqlBMDYYf", "s_phone": "16-668-570-1402", "s_comment": "tions haggle slyly about the sil" }
+{ "s_acctbal": 8102.62d, "s_name": "Supplier#000003347", "n_name": "UNITED KINGDOM", "p_partkey": 18344, "p_mfgr": "Manufacturer#5", "s_address": "m CtXS2S16i", "s_phone": "33-454-274-8532", "s_comment": "egrate with the slyly bold instructions. special foxes haggle silently among the" }
+{ "s_acctbal": 8046.07d, "s_name": "Supplier#000008780", "n_name": "FRANCE", "p_partkey": 191222, "p_mfgr": "Manufacturer#3", "s_address": "AczzuE0UK9osj ,Lx0Jmh", "s_phone": "16-473-215-6395", "s_comment": "onic platelets cajole after the regular instructions. permanently bold excuses" }
+{ "s_acctbal": 8042.09d, "s_name": "Supplier#000003245", "n_name": "RUSSIA", "p_partkey": 135705, "p_mfgr": "Manufacturer#4", "s_address": "Dh8Ikg39onrbOL4DyTfGw8a9oKUX3d9Y", "s_phone": "32-836-132-8872", "s_comment": "osits. packages cajole slyly. furiously regular deposits cajole slyly. q" }
+{ "s_acctbal": 8042.09d, "s_name": "Supplier#000003245", "n_name": "RUSSIA", "p_partkey": 150729, "p_mfgr": "Manufacturer#1", "s_address": "Dh8Ikg39onrbOL4DyTfGw8a9oKUX3d9Y", "s_phone": "32-836-132-8872", "s_comment": "osits. packages cajole slyly. furiously regular deposits cajole slyly. q" }
+{ "s_acctbal": 7992.4d, "s_name": "Supplier#000006108", "n_name": "FRANCE", "p_partkey": 118574, "p_mfgr": "Manufacturer#1", "s_address": "8tBydnTDwUqfBfFV4l3", "s_phone": "16-974-998-8937", "s_comment": " ironic ideas? fluffily even instructions wake. blithel" }
+{ "s_acctbal": 7980.65d, "s_name": "Supplier#000001288", "n_name": "FRANCE", "p_partkey": 13784, "p_mfgr": "Manufacturer#4", "s_address": "zE,7HgVPrCn", "s_phone": "16-646-464-8247", "s_comment": "ully bold courts. escapades nag slyly. furiously fluffy theodo" }
+{ "s_acctbal": 7950.37d, "s_name": "Supplier#000008101", "n_name": "GERMANY", "p_partkey": 33094, "p_mfgr": "Manufacturer#5", "s_address": "kkYvL6IuvojJgTNG IKkaXQDYgx8ILohj", "s_phone": "17-627-663-8014", "s_comment": "arefully unusual requests x-ray above the quickly final deposits. " }
+{ "s_acctbal": 7937.93d, "s_name": "Supplier#000009012", "n_name": "ROMANIA", "p_partkey": 83995, "p_mfgr": "Manufacturer#2", "s_address": "iUiTziH,Ek3i4lwSgunXMgrcTzwdb", "s_phone": "29-250-925-9690", "s_comment": "to the blithely ironic deposits nag sly" }
+{ "s_acctbal": 7914.45d, "s_name": "Supplier#000001013", "n_name": "RUSSIA", "p_partkey": 125988, "p_mfgr": "Manufacturer#2", "s_address": "riRcntps4KEDtYScjpMIWeYF6mNnR", "s_phone": "32-194-698-3365", "s_comment": " busily bold packages are dolphi" }
+{ "s_acctbal": 7912.91d, "s_name": "Supplier#000004211", "n_name": "GERMANY", "p_partkey": 159180, "p_mfgr": "Manufacturer#5", "s_address": "2wQRVovHrm3,v03IKzfTd,1PYsFXQFFOG", "s_phone": "17-266-947-7315", "s_comment": "ay furiously regular platelets. cou" }
+{ "s_acctbal": 7912.91d, "s_name": "Supplier#000004211", "n_name": "GERMANY", "p_partkey": 184210, "p_mfgr": "Manufacturer#4", "s_address": "2wQRVovHrm3,v03IKzfTd,1PYsFXQFFOG", "s_phone": "17-266-947-7315", "s_comment": "ay furiously regular platelets. cou" }
+{ "s_acctbal": 7894.56d, "s_name": "Supplier#000007981", "n_name": "GERMANY", "p_partkey": 85472, "p_mfgr": "Manufacturer#4", "s_address": "NSJ96vMROAbeXP", "s_phone": "17-963-404-3760", "s_comment": "ic platelets affix after the furiously" }
+{ "s_acctbal": 7887.08d, "s_name": "Supplier#000009792", "n_name": "GERMANY", "p_partkey": 164759, "p_mfgr": "Manufacturer#3", "s_address": "Y28ITVeYriT3kIGdV2K8fSZ V2UqT5H1Otz", "s_phone": "17-988-938-4296", "s_comment": "ckly around the carefully fluffy theodolites. slyly ironic pack" }
+{ "s_acctbal": 7871.5d, "s_name": "Supplier#000007206", "n_name": "RUSSIA", "p_partkey": 104695, "p_mfgr": "Manufacturer#1", "s_address": "3w fNCnrVmvJjE95sgWZzvW", "s_phone": "32-432-452-7731", "s_comment": "ironic requests. furiously final theodolites cajole. final, express packages sleep. quickly reg" }
+{ "s_acctbal": 7852.45d, "s_name": "Supplier#000005864", "n_name": "RUSSIA", "p_partkey": 8363, "p_mfgr": "Manufacturer#4", "s_address": "WCNfBPZeSXh3h,c", "s_phone": "32-454-883-3821", "s_comment": "usly unusual pinto beans. brave ideas sleep carefully quickly ironi" }
+{ "s_acctbal": 7850.66d, "s_name": "Supplier#000001518", "n_name": "UNITED KINGDOM", "p_partkey": 86501, "p_mfgr": "Manufacturer#1", "s_address": "ONda3YJiHKJOC", "s_phone": "33-730-383-3892", "s_comment": "ifts haggle fluffily pending pai" }
+{ "s_acctbal": 7843.52d, "s_name": "Supplier#000006683", "n_name": "FRANCE", "p_partkey": 11680, "p_mfgr": "Manufacturer#4", "s_address": "2Z0JGkiv01Y00oCFwUGfviIbhzCdy", "s_phone": "16-464-517-8943", "s_comment": " express, final pinto beans x-ray slyly asymptotes. unusual, unusual" }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q03_shipping_priority_nt.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q03_shipping_priority_nt.3.query.adm
new file mode 100644
index 0000000..1bb628e
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q03_shipping_priority_nt.3.query.adm
@@ -0,0 +1,10 @@
+{ "l_orderkey": 2456423, "revenue": 406181.0111d, "o_orderdate": "1995-03-05", "o_shippriority": 0 }
+{ "l_orderkey": 3459808, "revenue": 405838.69889999996d, "o_orderdate": "1995-03-04", "o_shippriority": 0 }
+{ "l_orderkey": 492164, "revenue": 390324.061d, "o_orderdate": "1995-02-19", "o_shippriority": 0 }
+{ "l_orderkey": 1188320, "revenue": 384537.9359d, "o_orderdate": "1995-03-09", "o_shippriority": 0 }
+{ "l_orderkey": 2435712, "revenue": 378673.0558d, "o_orderdate": "1995-02-26", "o_shippriority": 0 }
+{ "l_orderkey": 4878020, "revenue": 378376.7952d, "o_orderdate": "1995-03-12", "o_shippriority": 0 }
+{ "l_orderkey": 5521732, "revenue": 375153.92150000005d, "o_orderdate": "1995-03-13", "o_shippriority": 0 }
+{ "l_orderkey": 2628192, "revenue": 373133.30939999997d, "o_orderdate": "1995-02-22", "o_shippriority": 0 }
+{ "l_orderkey": 993600, "revenue": 371407.4595d, "o_orderdate": "1995-03-05", "o_shippriority": 0 }
+{ "l_orderkey": 2300070, "revenue": 367371.1452d, "o_orderdate": "1995-03-13", "o_shippriority": 0 }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q04_order_priority.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q04_order_priority.3.query.adm
new file mode 100644
index 0000000..365c1d6
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q04_order_priority.3.query.adm
@@ -0,0 +1,5 @@
+{ "order_priority": "1-URGENT", "count": 10594i64 }
+{ "order_priority": "2-HIGH", "count": 10476i64 }
+{ "order_priority": "3-MEDIUM", "count": 10410i64 }
+{ "order_priority": "4-NOT SPECIFIED", "count": 10556i64 }
+{ "order_priority": "5-LOW", "count": 10487i64 }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q06_forecast_revenue_change.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q06_forecast_revenue_change.3.query.adm
new file mode 100644
index 0000000..e80bbac
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q06_forecast_revenue_change.3.query.adm
@@ -0,0 +1 @@
+{ "revenue": 1.231410782283004E8d }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q07_volume_shipping.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q07_volume_shipping.3.query.adm
new file mode 100644
index 0000000..9ca7ac2
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q07_volume_shipping.3.query.adm
@@ -0,0 +1,4 @@
+{ "supp_nation": "FRANCE", "cust_nation": "GERMANY", "l_year": 1995, "revenue": 5.463973273359997E7d }
+{ "supp_nation": "FRANCE", "cust_nation": "GERMANY", "l_year": 1996, "revenue": 5.4633083307599954E7d }
+{ "supp_nation": "GERMANY", "cust_nation": "FRANCE", "l_year": 1995, "revenue": 5.253174666970002E7d }
+{ "supp_nation": "GERMANY", "cust_nation": "FRANCE", "l_year": 1996, "revenue": 5.252054902239985E7d }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q10_returned_item.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q10_returned_item.3.query.adm
new file mode 100644
index 0000000..9c45208
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q10_returned_item.3.query.adm
@@ -0,0 +1,20 @@
+{ "c_custkey": 57040, "c_name": "Customer#000057040", "revenue": 734235.2455d, "c_acctbal": 632.87d, "n_name": "JAPAN", "c_address": "Eioyzjf4pp", "c_phone": "22-895-641-3466", "c_comment": "sits. slyly regular requests sleep alongside of the regular inst" }
+{ "c_custkey": 143347, "c_name": "Customer#000143347", "revenue": 721002.6947999999d, "c_acctbal": 2557.47d, "n_name": "EGYPT", "c_address": "1aReFYv,Kw4", "c_phone": "14-742-935-3718", "c_comment": "ggle carefully enticing requests. final deposits use bold, bold pinto beans. ironic, idle re" }
+{ "c_custkey": 60838, "c_name": "Customer#000060838", "revenue": 679127.3076999999d, "c_acctbal": 2454.77d, "n_name": "BRAZIL", "c_address": "64EaJ5vMAHWJlBOxJklpNc2RJiWE", "c_phone": "12-913-494-9813", "c_comment": " need to boost against the slyly regular account" }
+{ "c_custkey": 101998, "c_name": "Customer#000101998", "revenue": 637029.5667d, "c_acctbal": 3790.89d, "n_name": "UNITED KINGDOM", "c_address": "01c9CILnNtfOQYmZj", "c_phone": "33-593-865-6378", "c_comment": "ress foxes wake slyly after the bold excuses. ironic platelets are furiously carefully bold theodolites" }
+{ "c_custkey": 125341, "c_name": "Customer#000125341", "revenue": 633508.0860000001d, "c_acctbal": 4983.51d, "n_name": "GERMANY", "c_address": "S29ODD6bceU8QSuuEJznkNaK", "c_phone": "17-582-695-5962", "c_comment": "arefully even depths. blithely even excuses sleep furiously. foxes use except the dependencies. ca" }
+{ "c_custkey": 25501, "c_name": "Customer#000025501", "revenue": 620269.7848999999d, "c_acctbal": 7725.04d, "n_name": "ETHIOPIA", "c_address": " W556MXuoiaYCCZamJI,Rn0B4ACUGdkQ8DZ", "c_phone": "15-874-808-6793", "c_comment": "he pending instructions wake carefully at the pinto beans. regular, final instructions along the slyly fina" }
+{ "c_custkey": 115831, "c_name": "Customer#000115831", "revenue": 596423.8672000001d, "c_acctbal": 5098.1d, "n_name": "FRANCE", "c_address": "rFeBbEEyk dl ne7zV5fDrmiq1oK09wV7pxqCgIc", "c_phone": "16-715-386-3788", "c_comment": "l somas sleep. furiously final deposits wake blithely regular pinto b" }
+{ "c_custkey": 84223, "c_name": "Customer#000084223", "revenue": 594998.0238999999d, "c_acctbal": 528.65d, "n_name": "UNITED KINGDOM", "c_address": "nAVZCs6BaWap rrM27N 2qBnzc5WBauxbA", "c_phone": "33-442-824-8191", "c_comment": " slyly final deposits haggle regular, pending dependencies. pending escapades wake " }
+{ "c_custkey": 54289, "c_name": "Customer#000054289", "revenue": 585603.3918000001d, "c_acctbal": 5583.02d, "n_name": "IRAN", "c_address": "vXCxoCsU0Bad5JQI ,oobkZ", "c_phone": "20-834-292-4707", "c_comment": "ely special foxes are quickly finally ironic p" }
+{ "c_custkey": 39922, "c_name": "Customer#000039922", "revenue": 584878.1133999999d, "c_acctbal": 7321.11d, "n_name": "GERMANY", "c_address": "Zgy4s50l2GKN4pLDPBU8m342gIw6R", "c_phone": "17-147-757-8036", "c_comment": "y final requests. furiously final foxes cajole blithely special platelets. f" }
+{ "c_custkey": 6226, "c_name": "Customer#000006226", "revenue": 576783.7606d, "c_acctbal": 2230.09d, "n_name": "UNITED KINGDOM", "c_address": "8gPu8,NPGkfyQQ0hcIYUGPIBWc,ybP5g,", "c_phone": "33-657-701-3391", "c_comment": "ending platelets along the express deposits cajole carefully final " }
+{ "c_custkey": 922, "c_name": "Customer#000000922", "revenue": 576767.5332999999d, "c_acctbal": 3869.25d, "n_name": "GERMANY", "c_address": "Az9RFaut7NkPnc5zSD2PwHgVwr4jRzq", "c_phone": "17-945-916-9648", "c_comment": "luffily fluffy deposits. packages c" }
+{ "c_custkey": 147946, "c_name": "Customer#000147946", "revenue": 576455.132d, "c_acctbal": 2030.13d, "n_name": "ALGERIA", "c_address": "iANyZHjqhyy7Ajah0pTrYyhJ", "c_phone": "10-886-956-3143", "c_comment": "ithely ironic deposits haggle blithely ironic requests. quickly regu" }
+{ "c_custkey": 115640, "c_name": "Customer#000115640", "revenue": 569341.1933d, "c_acctbal": 6436.1d, "n_name": "ARGENTINA", "c_address": "Vtgfia9qI 7EpHgecU1X", "c_phone": "11-411-543-4901", "c_comment": "ost slyly along the patterns; pinto be" }
+{ "c_custkey": 73606, "c_name": "Customer#000073606", "revenue": 568656.8578d, "c_acctbal": 1785.67d, "n_name": "JAPAN", "c_address": "xuR0Tro5yChDfOCrjkd2ol", "c_phone": "22-437-653-6966", "c_comment": "he furiously regular ideas. slowly" }
+{ "c_custkey": 110246, "c_name": "Customer#000110246", "revenue": 566842.9815d, "c_acctbal": 7763.35d, "n_name": "VIETNAM", "c_address": "7KzflgX MDOq7sOkI", "c_phone": "31-943-426-9837", "c_comment": "egular deposits serve blithely above the fl" }
+{ "c_custkey": 142549, "c_name": "Customer#000142549", "revenue": 563537.2368d, "c_acctbal": 5085.99d, "n_name": "INDONESIA", "c_address": "ChqEoK43OysjdHbtKCp6dKqjNyvvi9", "c_phone": "19-955-562-2398", "c_comment": "sleep pending courts. ironic deposits against the carefully unusual platelets cajole carefully express accounts." }
+{ "c_custkey": 146149, "c_name": "Customer#000146149", "revenue": 557254.9865d, "c_acctbal": 1791.55d, "n_name": "ROMANIA", "c_address": "s87fvzFQpU", "c_phone": "29-744-164-6487", "c_comment": " of the slyly silent accounts. quickly final accounts across the " }
+{ "c_custkey": 52528, "c_name": "Customer#000052528", "revenue": 556397.3509000001d, "c_acctbal": 551.79d, "n_name": "ARGENTINA", "c_address": "NFztyTOR10UOJ", "c_phone": "11-208-192-3205", "c_comment": " deposits hinder. blithely pending asymptotes breach slyly regular re" }
+{ "c_custkey": 23431, "c_name": "Customer#000023431", "revenue": 554269.536d, "c_acctbal": 3381.86d, "n_name": "ROMANIA", "c_address": "HgiV0phqhaIa9aydNoIlb", "c_phone": "29-915-458-2654", "c_comment": "nusual, even instructions: furiously stealthy n" }
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q11_important_stock.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q11_important_stock.3.query.adm
new file mode 100644
index 0000000..4c48b49
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q11_important_stock.3.query.adm
@@ -0,0 +1,1048 @@
+{ "partkey": 129760, "part_value": 1.753845686E7d }
+{ "partkey": 166726, "part_value": 1.650335392E7d }
+{ "partkey": 191287, "part_value": 1.6474801969999999E7d }
+{ "partkey": 161758, "part_value": 1.610175554E7d }
+{ "partkey": 34452, "part_value": 1.598384472E7d }
+{ "partkey": 139035, "part_value": 1.590707834E7d }
+{ "partkey": 9403, "part_value": 1.5451755620000001E7d }
+{ "partkey": 154358, "part_value": 1.5212937879999999E7d }
+{ "partkey": 38823, "part_value": 1.506480286E7d }
+{ "partkey": 85606, "part_value": 1.5053957150000002E7d }
+{ "partkey": 33354, "part_value": 1.44082974E7d }
+{ "partkey": 154747, "part_value": 1.440758068E7d }
+{ "partkey": 82865, "part_value": 1.4235489780000001E7d }
+{ "partkey": 76094, "part_value": 1.409424704E7d }
+{ "partkey": 222, "part_value": 1.393777774E7d }
+{ "partkey": 121271, "part_value": 1.3908336E7d }
+{ "partkey": 55221, "part_value": 1.371612047E7d }
+{ "partkey": 22819, "part_value": 1.3666434280000001E7d }
+{ "partkey": 76281, "part_value": 1.364685368E7d }
+{ "partkey": 85298, "part_value": 1.358115493E7d }
+{ "partkey": 85158, "part_value": 1.3554904E7d }
+{ "partkey": 139684, "part_value": 1.3535538719999999E7d }
+{ "partkey": 31034, "part_value": 1.349802525E7d }
+{ "partkey": 87305, "part_value": 1.348284704E7d }
+{ "partkey": 10181, "part_value": 1.344514875E7d }
+{ "partkey": 62323, "part_value": 1.34118243E7d }
+{ "partkey": 26489, "part_value": 1.337725638E7d }
+{ "partkey": 96493, "part_value": 1.333905783E7d }
+{ "partkey": 56548, "part_value": 1.3329014969999999E7d }
+{ "partkey": 55576, "part_value": 1.3306843350000001E7d }
+{ "partkey": 159751, "part_value": 1.330661448E7d }
+{ "partkey": 92406, "part_value": 1.32874145E7d }
+{ "partkey": 182636, "part_value": 1.3223726739999998E7d }
+{ "partkey": 199969, "part_value": 1.313528821E7d }
+{ "partkey": 62865, "part_value": 1.3001926940000001E7d }
+{ "partkey": 7284, "part_value": 1.2945298190000001E7d }
+{ "partkey": 197867, "part_value": 1.294451052E7d }
+{ "partkey": 11562, "part_value": 1.293157551E7d }
+{ "partkey": 75165, "part_value": 1.2916918120000001E7d }
+{ "partkey": 97175, "part_value": 1.29112835E7d }
+{ "partkey": 140840, "part_value": 1.289656223E7d }
+{ "partkey": 65241, "part_value": 1.289060046E7d }
+{ "partkey": 166120, "part_value": 1.287692722E7d }
+{ "partkey": 9035, "part_value": 1.28638287E7d }
+{ "partkey": 144616, "part_value": 1.2853549299999999E7d }
+{ "partkey": 176723, "part_value": 1.283230974E7d }
+{ "partkey": 170884, "part_value": 1.2792136579999998E7d }
+{ "partkey": 29790, "part_value": 1.272330033E7d }
+{ "partkey": 95213, "part_value": 1.255548373E7d }
+{ "partkey": 183873, "part_value": 1.255053305E7d }
+{ "partkey": 171235, "part_value": 1.24765383E7d }
+{ "partkey": 21533, "part_value": 1.243782132E7d }
+{ "partkey": 17290, "part_value": 1.24321595E7d }
+{ "partkey": 156397, "part_value": 1.22606235E7d }
+{ "partkey": 122611, "part_value": 1.222281298E7d }
+{ "partkey": 139155, "part_value": 1.222031925E7d }
+{ "partkey": 146316, "part_value": 1.221580061E7d }
+{ "partkey": 171381, "part_value": 1.219973452E7d }
+{ "partkey": 198633, "part_value": 1.207822695E7d }
+{ "partkey": 167417, "part_value": 1.2046637620000001E7d }
+{ "partkey": 59512, "part_value": 1.204346876E7d }
+{ "partkey": 31688, "part_value": 1.203489364E7d }
+{ "partkey": 159586, "part_value": 1.2001505840000002E7d }
+{ "partkey": 8993, "part_value": 1.19638143E7d }
+{ "partkey": 120302, "part_value": 1.1857707549999999E7d }
+{ "partkey": 43536, "part_value": 1.177934052E7d }
+{ "partkey": 9552, "part_value": 1.177690916E7d }
+{ "partkey": 86223, "part_value": 1.177220508E7d }
+{ "partkey": 53776, "part_value": 1.175866965E7d }
+{ "partkey": 131285, "part_value": 1.161695374E7d }
+{ "partkey": 91628, "part_value": 1.161111483E7d }
+{ "partkey": 169644, "part_value": 1.1567959719999999E7d }
+{ "partkey": 182299, "part_value": 1.156746205E7d }
+{ "partkey": 33107, "part_value": 1.145381876E7d }
+{ "partkey": 104184, "part_value": 1.1436657440000001E7d }
+{ "partkey": 67027, "part_value": 1.141912714E7d }
+{ "partkey": 176869, "part_value": 1.1371451709999999E7d }
+{ "partkey": 30885, "part_value": 1.136967479E7d }
+{ "partkey": 54420, "part_value": 1.1345076879999999E7d }
+{ "partkey": 72240, "part_value": 1.1313951049999999E7d }
+{ "partkey": 178708, "part_value": 1.1294635169999998E7d }
+{ "partkey": 81298, "part_value": 1.127368613E7d }
+{ "partkey": 158324, "part_value": 1.124344272E7d }
+{ "partkey": 117095, "part_value": 1.124253524E7d }
+{ "partkey": 176793, "part_value": 1.1237733379999999E7d }
+{ "partkey": 86091, "part_value": 1.1177793790000001E7d }
+{ "partkey": 116033, "part_value": 1.114543436E7d }
+{ "partkey": 129058, "part_value": 1.11191122E7d }
+{ "partkey": 193714, "part_value": 1.110470639E7d }
+{ "partkey": 117195, "part_value": 1.107721796E7d }
+{ "partkey": 49851, "part_value": 1.1043701780000001E7d }
+{ "partkey": 19791, "part_value": 1.103066262E7d }
+{ "partkey": 75800, "part_value": 1.1012401620000001E7d }
+{ "partkey": 161562, "part_value": 1.0996371690000001E7d }
+{ "partkey": 10119, "part_value": 1.098001575E7d }
+{ "partkey": 39185, "part_value": 1.0970042559999999E7d }
+{ "partkey": 47223, "part_value": 1.0950022129999999E7d }
+{ "partkey": 175594, "part_value": 1.094292305E7d }
+{ "partkey": 111295, "part_value": 1.089367561E7d }
+{ "partkey": 155446, "part_value": 1.085276457E7d }
+{ "partkey": 156391, "part_value": 1.0839810379999999E7d }
+{ "partkey": 40884, "part_value": 1.0837234190000001E7d }
+{ "partkey": 141288, "part_value": 1.083713021E7d }
+{ "partkey": 152388, "part_value": 1.083097782E7d }
+{ "partkey": 33449, "part_value": 1.083085872E7d }
+{ "partkey": 149035, "part_value": 1.082613002E7d }
+{ "partkey": 162620, "part_value": 1.081427568E7d }
+{ "partkey": 118324, "part_value": 1.07917881E7d }
+{ "partkey": 38932, "part_value": 1.077754175E7d }
+{ "partkey": 121294, "part_value": 1.0764225219999999E7d }
+{ "partkey": 48721, "part_value": 1.076258249E7d }
+{ "partkey": 63342, "part_value": 1.07401326E7d }
+{ "partkey": 5614, "part_value": 1.0724668799999999E7d }
+{ "partkey": 62266, "part_value": 1.07111431E7d }
+{ "partkey": 100202, "part_value": 1.069667555E7d }
+{ "partkey": 197741, "part_value": 1.0688560719999999E7d }
+{ "partkey": 169178, "part_value": 1.06485228E7d }
+{ "partkey": 5271, "part_value": 1.063939265E7d }
+{ "partkey": 34499, "part_value": 1.0584177100000001E7d }
+{ "partkey": 71108, "part_value": 1.056911756E7d }
+{ "partkey": 137132, "part_value": 1.0539880469999999E7d }
+{ "partkey": 78451, "part_value": 1.052487324E7d }
+{ "partkey": 150827, "part_value": 1.050381048E7d }
+{ "partkey": 107237, "part_value": 1.048803084E7d }
+{ "partkey": 101727, "part_value": 1.0473558100000001E7d }
+{ "partkey": 58708, "part_value": 1.046628044E7d }
+{ "partkey": 89768, "part_value": 1.0465477219999999E7d }
+{ "partkey": 146493, "part_value": 1.044429158E7d }
+{ "partkey": 55424, "part_value": 1.044400648E7d }
+{ "partkey": 16560, "part_value": 1.042557474E7d }
+{ "partkey": 133114, "part_value": 1.04150979E7d }
+{ "partkey": 195810, "part_value": 1.04136252E7d }
+{ "partkey": 76673, "part_value": 1.039197718E7d }
+{ "partkey": 97305, "part_value": 1.039089057E7d }
+{ "partkey": 134210, "part_value": 1.038721002E7d }
+{ "partkey": 188536, "part_value": 1.038652992E7d }
+{ "partkey": 122255, "part_value": 1.033576032E7d }
+{ "partkey": 2682, "part_value": 1.03129661E7d }
+{ "partkey": 43814, "part_value": 1.030308661E7d }
+{ "partkey": 34767, "part_value": 1.029040518E7d }
+{ "partkey": 165584, "part_value": 1.027370589E7d }
+{ "partkey": 2231, "part_value": 1.027041555E7d }
+{ "partkey": 111259, "part_value": 1.026325656E7d }
+{ "partkey": 195578, "part_value": 1.023979582E7d }
+{ "partkey": 21093, "part_value": 1.02175313E7d }
+{ "partkey": 29856, "part_value": 1.0216932540000001E7d }
+{ "partkey": 133686, "part_value": 1.021334576E7d }
+{ "partkey": 87745, "part_value": 1.01855094E7d }
+{ "partkey": 135153, "part_value": 1.01793797E7d }
+{ "partkey": 11773, "part_value": 1.016741084E7d }
+{ "partkey": 76316, "part_value": 1.01651517E7d }
+{ "partkey": 123076, "part_value": 1.016122578E7d }
+{ "partkey": 91894, "part_value": 1.013046219E7d }
+{ "partkey": 39741, "part_value": 1.012838752E7d }
+{ "partkey": 111753, "part_value": 1.0119780979999999E7d }
+{ "partkey": 142729, "part_value": 1.010474889E7d }
+{ "partkey": 116775, "part_value": 1.0097750419999998E7d }
+{ "partkey": 102589, "part_value": 1.003478436E7d }
+{ "partkey": 186268, "part_value": 1.001218157E7d }
+{ "partkey": 44545, "part_value": 1.000028648E7d }
+{ "partkey": 23307, "part_value": 9966577.5d }
+{ "partkey": 124281, "part_value": 9930018.9d }
+{ "partkey": 69604, "part_value": 9925730.64d }
+{ "partkey": 21971, "part_value": 9908982.030000001d }
+{ "partkey": 58148, "part_value": 9895894.4d }
+{ "partkey": 16532, "part_value": 9886529.9d }
+{ "partkey": 159180, "part_value": 9883744.43d }
+{ "partkey": 74733, "part_value": 9877582.879999999d }
+{ "partkey": 35173, "part_value": 9858275.92d }
+{ "partkey": 7116, "part_value": 9856881.020000001d }
+{ "partkey": 124620, "part_value": 9838589.14d }
+{ "partkey": 122108, "part_value": 9829949.350000001d }
+{ "partkey": 67200, "part_value": 9828690.69d }
+{ "partkey": 164775, "part_value": 9821424.44d }
+{ "partkey": 9039, "part_value": 9816447.72d }
+{ "partkey": 14912, "part_value": 9803102.2d }
+{ "partkey": 190906, "part_value": 9791315.7d }
+{ "partkey": 130398, "part_value": 9781674.27d }
+{ "partkey": 119310, "part_value": 9776927.209999999d }
+{ "partkey": 10132, "part_value": 9770930.780000001d }
+{ "partkey": 107211, "part_value": 9757586.25d }
+{ "partkey": 113958, "part_value": 9757065.5d }
+{ "partkey": 37009, "part_value": 9748362.69d }
+{ "partkey": 66746, "part_value": 9743528.76d }
+{ "partkey": 134486, "part_value": 9731922.0d }
+{ "partkey": 15945, "part_value": 9731096.45d }
+{ "partkey": 55307, "part_value": 9717745.8d }
+{ "partkey": 56362, "part_value": 9714922.83d }
+{ "partkey": 57726, "part_value": 9711792.100000001d }
+{ "partkey": 57256, "part_value": 9708621.0d }
+{ "partkey": 112292, "part_value": 9701653.08d }
+{ "partkey": 87514, "part_value": 9699492.53d }
+{ "partkey": 174206, "part_value": 9680562.02d }
+{ "partkey": 72865, "part_value": 9679043.34d }
+{ "partkey": 114357, "part_value": 9671017.44d }
+{ "partkey": 112807, "part_value": 9665019.21d }
+{ "partkey": 115203, "part_value": 9661018.73d }
+{ "partkey": 177454, "part_value": 9658906.35d }
+{ "partkey": 161275, "part_value": 9634313.71d }
+{ "partkey": 61893, "part_value": 9617095.440000001d }
+{ "partkey": 122219, "part_value": 9604888.200000001d }
+{ "partkey": 183427, "part_value": 9601362.58d }
+{ "partkey": 59158, "part_value": 9599705.96d }
+{ "partkey": 61931, "part_value": 9584918.98d }
+{ "partkey": 5532, "part_value": 9579964.14d }
+{ "partkey": 20158, "part_value": 9576714.38d }
+{ "partkey": 167199, "part_value": 9557413.08d }
+{ "partkey": 38869, "part_value": 9550279.53d }
+{ "partkey": 86949, "part_value": 9541943.700000001d }
+{ "partkey": 198544, "part_value": 9538613.92d }
+{ "partkey": 193762, "part_value": 9538238.94d }
+{ "partkey": 108807, "part_value": 9536247.16d }
+{ "partkey": 168324, "part_value": 9535647.99d }
+{ "partkey": 115588, "part_value": 9532195.04d }
+{ "partkey": 141372, "part_value": 9529702.14d }
+{ "partkey": 175120, "part_value": 9526068.66d }
+{ "partkey": 163851, "part_value": 9522808.83d }
+{ "partkey": 160954, "part_value": 9520359.45d }
+{ "partkey": 117757, "part_value": 9517882.8d }
+{ "partkey": 52594, "part_value": 9508325.76d }
+{ "partkey": 60960, "part_value": 9498843.06d }
+{ "partkey": 70272, "part_value": 9495775.620000001d }
+{ "partkey": 44050, "part_value": 9495515.36d }
+{ "partkey": 152213, "part_value": 9494756.96d }
+{ "partkey": 121203, "part_value": 9492601.3d }
+{ "partkey": 70114, "part_value": 9491012.3d }
+{ "partkey": 167588, "part_value": 9484741.11d }
+{ "partkey": 136455, "part_value": 9476241.78d }
+{ "partkey": 4357, "part_value": 9464355.64d }
+{ "partkey": 6786, "part_value": 9463632.57d }
+{ "partkey": 61345, "part_value": 9455336.700000001d }
+{ "partkey": 160826, "part_value": 9446754.84d }
+{ "partkey": 71275, "part_value": 9440138.4d }
+{ "partkey": 77746, "part_value": 9439118.35d }
+{ "partkey": 91289, "part_value": 9437472.0d }
+{ "partkey": 56723, "part_value": 9435102.16d }
+{ "partkey": 86647, "part_value": 9434604.18d }
+{ "partkey": 131234, "part_value": 9432120.0d }
+{ "partkey": 198129, "part_value": 9427651.36d }
+{ "partkey": 165530, "part_value": 9426193.68d }
+{ "partkey": 69233, "part_value": 9425053.92d }
+{ "partkey": 6243, "part_value": 9423304.66d }
+{ "partkey": 90110, "part_value": 9420422.7d }
+{ "partkey": 191980, "part_value": 9419368.360000001d }
+{ "partkey": 38461, "part_value": 9419316.07d }
+{ "partkey": 167873, "part_value": 9419024.489999998d }
+{ "partkey": 159373, "part_value": 9416950.149999999d }
+{ "partkey": 128707, "part_value": 9413428.5d }
+{ "partkey": 45267, "part_value": 9410863.78d }
+{ "partkey": 48460, "part_value": 9409793.93d }
+{ "partkey": 197672, "part_value": 9406887.68d }
+{ "partkey": 60884, "part_value": 9403442.4d }
+{ "partkey": 15209, "part_value": 9403245.31d }
+{ "partkey": 138049, "part_value": 9401262.1d }
+{ "partkey": 199286, "part_value": 9391770.7d }
+{ "partkey": 19629, "part_value": 9391236.4d }
+{ "partkey": 134019, "part_value": 9390615.15d }
+{ "partkey": 169475, "part_value": 9387639.58d }
+{ "partkey": 165918, "part_value": 9379510.44d }
+{ "partkey": 135602, "part_value": 9374251.540000001d }
+{ "partkey": 162323, "part_value": 9367566.51d }
+{ "partkey": 96277, "part_value": 9360850.68d }
+{ "partkey": 98336, "part_value": 9359671.290000001d }
+{ "partkey": 119781, "part_value": 9356395.73d }
+{ "partkey": 34440, "part_value": 9355365.0d }
+{ "partkey": 57362, "part_value": 9355180.1d }
+{ "partkey": 167236, "part_value": 9352973.84d }
+{ "partkey": 38463, "part_value": 9347530.940000001d }
+{ "partkey": 86749, "part_value": 9346826.44d }
+{ "partkey": 170007, "part_value": 9345699.9d }
+{ "partkey": 193087, "part_value": 9343744.0d }
+{ "partkey": 150383, "part_value": 9332576.75d }
+{ "partkey": 60932, "part_value": 9329582.02d }
+{ "partkey": 128420, "part_value": 9328206.35d }
+{ "partkey": 162145, "part_value": 9327722.88d }
+{ "partkey": 55686, "part_value": 9320304.4d }
+{ "partkey": 163080, "part_value": 9304916.96d }
+{ "partkey": 160583, "part_value": 9303515.92d }
+{ "partkey": 118153, "part_value": 9298606.56d }
+{ "partkey": 152634, "part_value": 9282184.57d }
+{ "partkey": 84731, "part_value": 9276586.919999998d }
+{ "partkey": 119989, "part_value": 9273814.2d }
+{ "partkey": 114584, "part_value": 9269698.65d }
+{ "partkey": 131817, "part_value": 9268570.08d }
+{ "partkey": 29068, "part_value": 9256583.88d }
+{ "partkey": 44116, "part_value": 9255922.0d }
+{ "partkey": 115818, "part_value": 9253311.91d }
+{ "partkey": 103388, "part_value": 9239218.08d }
+{ "partkey": 186118, "part_value": 9236209.12d }
+{ "partkey": 155809, "part_value": 9235410.84d }
+{ "partkey": 147003, "part_value": 9234847.99d }
+{ "partkey": 27769, "part_value": 9232511.64d }
+{ "partkey": 112779, "part_value": 9231927.360000001d }
+{ "partkey": 124851, "part_value": 9228982.68d }
+{ "partkey": 158488, "part_value": 9227216.4d }
+{ "partkey": 83328, "part_value": 9224792.2d }
+{ "partkey": 136797, "part_value": 9222927.09d }
+{ "partkey": 141730, "part_value": 9216370.68d }
+{ "partkey": 87304, "part_value": 9215695.5d }
+{ "partkey": 156004, "part_value": 9215557.9d }
+{ "partkey": 140740, "part_value": 9215329.200000001d }
+{ "partkey": 100648, "part_value": 9212185.08d }
+{ "partkey": 174774, "part_value": 9211718.0d }
+{ "partkey": 37644, "part_value": 9211578.6d }
+{ "partkey": 48807, "part_value": 9209496.24d }
+{ "partkey": 95940, "part_value": 9207948.4d }
+{ "partkey": 141586, "part_value": 9206699.22d }
+{ "partkey": 147248, "part_value": 9205654.95d }
+{ "partkey": 61372, "part_value": 9205228.76d }
+{ "partkey": 52970, "part_value": 9204415.950000001d }
+{ "partkey": 26430, "part_value": 9203710.51d }
+{ "partkey": 28504, "part_value": 9201669.2d }
+{ "partkey": 25810, "part_value": 9198878.5d }
+{ "partkey": 125329, "part_value": 9198688.5d }
+{ "partkey": 167867, "part_value": 9194022.719999999d }
+{ "partkey": 134767, "part_value": 9191444.72d }
+{ "partkey": 127745, "part_value": 9191271.559999999d }
+{ "partkey": 69208, "part_value": 9187110.0d }
+{ "partkey": 155222, "part_value": 9186469.16d }
+{ "partkey": 196916, "part_value": 9182995.82d }
+{ "partkey": 195590, "part_value": 9176353.12d }
+{ "partkey": 169155, "part_value": 9175176.09d }
+{ "partkey": 81558, "part_value": 9171946.5d }
+{ "partkey": 185136, "part_value": 9171293.04d }
+{ "partkey": 114790, "part_value": 9168509.1d }
+{ "partkey": 194142, "part_value": 9165836.61d }
+{ "partkey": 167639, "part_value": 9161165.0d }
+{ "partkey": 11241, "part_value": 9160789.46d }
+{ "partkey": 82628, "part_value": 9160155.54d }
+{ "partkey": 41399, "part_value": 9148338.0d }
+{ "partkey": 30755, "part_value": 9146196.84d }
+{ "partkey": 6944, "part_value": 9143574.58d }
+{ "partkey": 6326, "part_value": 9138803.16d }
+{ "partkey": 101296, "part_value": 9135657.620000001d }
+{ "partkey": 181479, "part_value": 9121093.299999999d }
+{ "partkey": 76898, "part_value": 9120983.1d }
+{ "partkey": 64274, "part_value": 9118745.25d }
+{ "partkey": 175826, "part_value": 9117387.99d }
+{ "partkey": 142215, "part_value": 9116876.88d }
+{ "partkey": 103415, "part_value": 9113128.620000001d }
+{ "partkey": 119765, "part_value": 9110768.79d }
+{ "partkey": 107624, "part_value": 9108837.45d }
+{ "partkey": 84215, "part_value": 9105257.36d }
+{ "partkey": 73774, "part_value": 9102651.92d }
+{ "partkey": 173972, "part_value": 9102069.0d }
+{ "partkey": 69817, "part_value": 9095513.879999999d }
+{ "partkey": 86943, "part_value": 9092253.0d }
+{ "partkey": 138859, "part_value": 9087719.3d }
+{ "partkey": 162273, "part_value": 9085296.48d }
+{ "partkey": 175945, "part_value": 9080401.21d }
+{ "partkey": 16836, "part_value": 9075715.44d }
+{ "partkey": 70224, "part_value": 9075265.950000001d }
+{ "partkey": 139765, "part_value": 9074755.89d }
+{ "partkey": 30319, "part_value": 9073233.1d }
+{ "partkey": 3851, "part_value": 9072657.24d }
+{ "partkey": 181271, "part_value": 9070631.52d }
+{ "partkey": 162184, "part_value": 9068835.780000001d }
+{ "partkey": 81683, "part_value": 9067258.47d }
+{ "partkey": 153028, "part_value": 9067010.51d }
+{ "partkey": 123324, "part_value": 9061870.95d }
+{ "partkey": 186481, "part_value": 9058608.299999999d }
+{ "partkey": 167680, "part_value": 9052908.76d }
+{ "partkey": 165293, "part_value": 9050545.700000001d }
+{ "partkey": 122148, "part_value": 9046298.17d }
+{ "partkey": 138604, "part_value": 9045840.799999999d }
+{ "partkey": 78851, "part_value": 9044822.600000001d }
+{ "partkey": 137280, "part_value": 9042355.34d }
+{ "partkey": 8823, "part_value": 9040855.1d }
+{ "partkey": 163900, "part_value": 9040848.48d }
+{ "partkey": 75600, "part_value": 9035392.45d }
+{ "partkey": 81676, "part_value": 9031999.4d }
+{ "partkey": 46033, "part_value": 9031460.58d }
+{ "partkey": 194917, "part_value": 9028500.0d }
+{ "partkey": 133936, "part_value": 9026949.020000001d }
+{ "partkey": 33182, "part_value": 9024971.1d }
+{ "partkey": 34220, "part_value": 9021485.39d }
+{ "partkey": 20118, "part_value": 9019942.6d }
+{ "partkey": 178258, "part_value": 9019881.66d }
+{ "partkey": 15560, "part_value": 9017687.28d }
+{ "partkey": 111425, "part_value": 9016198.56d }
+{ "partkey": 95942, "part_value": 9015585.120000001d }
+{ "partkey": 132709, "part_value": 9015240.15d }
+{ "partkey": 39731, "part_value": 9014746.95d }
+{ "partkey": 154307, "part_value": 9012571.2d }
+{ "partkey": 23769, "part_value": 9008157.6d }
+{ "partkey": 93328, "part_value": 9007211.2d }
+{ "partkey": 142826, "part_value": 8998297.440000001d }
+{ "partkey": 188792, "part_value": 8996014.0d }
+{ "partkey": 68703, "part_value": 8994982.22d }
+{ "partkey": 145280, "part_value": 8990941.05d }
+{ "partkey": 150725, "part_value": 8985686.16d }
+{ "partkey": 172046, "part_value": 8982469.52d }
+{ "partkey": 70476, "part_value": 8967629.5d }
+{ "partkey": 124988, "part_value": 8966805.22d }
+{ "partkey": 17937, "part_value": 8963319.76d }
+{ "partkey": 177372, "part_value": 8954873.64d }
+{ "partkey": 137994, "part_value": 8950916.79d }
+{ "partkey": 84019, "part_value": 8950039.98d }
+{ "partkey": 40389, "part_value": 8946158.2d }
+{ "partkey": 69187, "part_value": 8941054.14d }
+{ "partkey": 4863, "part_value": 8939044.92d }
+{ "partkey": 50465, "part_value": 8930503.14d }
+{ "partkey": 43686, "part_value": 8915543.84d }
+{ "partkey": 131352, "part_value": 8909053.59d }
+{ "partkey": 198916, "part_value": 8906940.03d }
+{ "partkey": 135932, "part_value": 8905282.95d }
+{ "partkey": 104673, "part_value": 8903682.0d }
+{ "partkey": 152308, "part_value": 8903244.08d }
+{ "partkey": 135298, "part_value": 8900323.2d }
+{ "partkey": 156873, "part_value": 8899429.1d }
+{ "partkey": 157454, "part_value": 8897339.2d }
+{ "partkey": 75415, "part_value": 8897068.09d }
+{ "partkey": 46325, "part_value": 8895569.09d }
+{ "partkey": 1966, "part_value": 8895117.06d }
+{ "partkey": 24576, "part_value": 8895034.75d }
+{ "partkey": 19425, "part_value": 8890156.6d }
+{ "partkey": 169735, "part_value": 8890085.56d }
+{ "partkey": 32225, "part_value": 8889829.280000001d }
+{ "partkey": 124537, "part_value": 8889770.71d }
+{ "partkey": 146327, "part_value": 8887836.23d }
+{ "partkey": 121562, "part_value": 8887740.4d }
+{ "partkey": 44731, "part_value": 8882444.95d }
+{ "partkey": 93141, "part_value": 8881850.879999999d }
+{ "partkey": 187871, "part_value": 8873506.18d }
+{ "partkey": 71709, "part_value": 8873057.28d }
+{ "partkey": 151913, "part_value": 8869321.17d }
+{ "partkey": 33786, "part_value": 8868955.389999999d }
+{ "partkey": 35902, "part_value": 8868126.06d }
+{ "partkey": 23588, "part_value": 8867769.9d }
+{ "partkey": 24508, "part_value": 8867616.0d }
+{ "partkey": 161282, "part_value": 8866661.43d }
+{ "partkey": 188061, "part_value": 8862304.0d }
+{ "partkey": 132847, "part_value": 8862082.0d }
+{ "partkey": 166843, "part_value": 8861200.8d }
+{ "partkey": 30609, "part_value": 8860214.73d }
+{ "partkey": 56191, "part_value": 8856546.959999999d }
+{ "partkey": 160740, "part_value": 8852685.43d }
+{ "partkey": 71229, "part_value": 8846106.99d }
+{ "partkey": 91208, "part_value": 8845541.28d }
+{ "partkey": 10995, "part_value": 8845306.56d }
+{ "partkey": 78094, "part_value": 8839938.290000001d }
+{ "partkey": 36489, "part_value": 8838538.1d }
+{ "partkey": 198437, "part_value": 8836494.84d }
+{ "partkey": 151693, "part_value": 8833807.64d }
+{ "partkey": 185367, "part_value": 8829791.370000001d }
+{ "partkey": 65682, "part_value": 8820622.889999999d }
+{ "partkey": 65421, "part_value": 8819329.239999998d }
+{ "partkey": 122225, "part_value": 8816821.86d }
+{ "partkey": 85330, "part_value": 8811013.16d }
+{ "partkey": 64555, "part_value": 8810643.120000001d }
+{ "partkey": 104188, "part_value": 8808211.02d }
+{ "partkey": 54411, "part_value": 8805703.4d }
+{ "partkey": 39438, "part_value": 8805282.56d }
+{ "partkey": 70795, "part_value": 8800060.92d }
+{ "partkey": 20383, "part_value": 8799073.28d }
+{ "partkey": 21952, "part_value": 8798624.19d }
+{ "partkey": 63584, "part_value": 8796590.0d }
+{ "partkey": 158768, "part_value": 8796422.95d }
+{ "partkey": 166588, "part_value": 8796214.379999999d }
+{ "partkey": 120600, "part_value": 8793558.06d }
+{ "partkey": 157202, "part_value": 8788287.879999999d }
+{ "partkey": 55358, "part_value": 8786820.75d }
+{ "partkey": 168322, "part_value": 8786670.73d }
+{ "partkey": 25143, "part_value": 8786324.8d }
+{ "partkey": 5368, "part_value": 8786274.14d }
+{ "partkey": 114025, "part_value": 8786201.12d }
+{ "partkey": 97744, "part_value": 8785315.940000001d }
+{ "partkey": 164327, "part_value": 8784503.860000001d }
+{ "partkey": 76542, "part_value": 8782613.28d }
+{ "partkey": 4731, "part_value": 8772846.7d }
+{ "partkey": 157590, "part_value": 8772006.45d }
+{ "partkey": 154276, "part_value": 8771733.91d }
+{ "partkey": 28705, "part_value": 8771576.64d }
+{ "partkey": 100226, "part_value": 8769455.0d }
+{ "partkey": 179195, "part_value": 8769185.16d }
+{ "partkey": 184355, "part_value": 8768118.05d }
+{ "partkey": 120408, "part_value": 8768011.120000001d }
+{ "partkey": 63145, "part_value": 8761991.959999999d }
+{ "partkey": 53135, "part_value": 8753491.799999999d }
+{ "partkey": 173071, "part_value": 8750508.8d }
+{ "partkey": 41087, "part_value": 8749436.79d }
+{ "partkey": 194830, "part_value": 8747438.4d }
+{ "partkey": 43496, "part_value": 8743359.299999999d }
+{ "partkey": 30235, "part_value": 8741611.0d }
+{ "partkey": 26391, "part_value": 8741399.64d }
+{ "partkey": 191816, "part_value": 8740258.719999999d }
+{ "partkey": 47616, "part_value": 8737229.68d }
+{ "partkey": 152101, "part_value": 8734432.76d }
+{ "partkey": 163784, "part_value": 8730514.34d }
+{ "partkey": 5134, "part_value": 8728424.639999999d }
+{ "partkey": 155241, "part_value": 8725429.86d }
+{ "partkey": 188814, "part_value": 8724182.4d }
+{ "partkey": 140782, "part_value": 8720378.75d }
+{ "partkey": 153141, "part_value": 8719407.51d }
+{ "partkey": 169373, "part_value": 8718609.06d }
+{ "partkey": 41335, "part_value": 8714773.8d }
+{ "partkey": 197450, "part_value": 8714617.32d }
+{ "partkey": 87004, "part_value": 8714017.79d }
+{ "partkey": 181804, "part_value": 8712257.76d }
+{ "partkey": 122814, "part_value": 8711119.14d }
+{ "partkey": 109939, "part_value": 8709193.16d }
+{ "partkey": 98094, "part_value": 8708780.04d }
+{ "partkey": 74630, "part_value": 8708040.75d }
+{ "partkey": 197291, "part_value": 8706519.09d }
+{ "partkey": 184173, "part_value": 8705467.450000001d }
+{ "partkey": 192175, "part_value": 8705411.12d }
+{ "partkey": 19471, "part_value": 8702536.12d }
+{ "partkey": 18052, "part_value": 8702155.700000001d }
+{ "partkey": 135560, "part_value": 8698137.72d }
+{ "partkey": 152791, "part_value": 8697325.8d }
+{ "partkey": 170953, "part_value": 8696909.19d }
+{ "partkey": 116137, "part_value": 8696687.17d }
+{ "partkey": 7722, "part_value": 8696589.4d }
+{ "partkey": 49788, "part_value": 8694846.71d }
+{ "partkey": 13252, "part_value": 8694822.42d }
+{ "partkey": 12633, "part_value": 8694559.36d }
+{ "partkey": 193438, "part_value": 8690426.72d }
+{ "partkey": 17326, "part_value": 8689329.16d }
+{ "partkey": 96124, "part_value": 8679794.58d }
+{ "partkey": 143802, "part_value": 8676626.48d }
+{ "partkey": 30389, "part_value": 8675826.6d }
+{ "partkey": 75250, "part_value": 8675257.14d }
+{ "partkey": 72613, "part_value": 8673524.94d }
+{ "partkey": 123520, "part_value": 8672456.25d }
+{ "partkey": 325, "part_value": 8667741.280000001d }
+{ "partkey": 167291, "part_value": 8667556.18d }
+{ "partkey": 150119, "part_value": 8663403.54d }
+{ "partkey": 88420, "part_value": 8663355.4d }
+{ "partkey": 179784, "part_value": 8653021.34d }
+{ "partkey": 130884, "part_value": 8651970.0d }
+{ "partkey": 172611, "part_value": 8648217.0d }
+{ "partkey": 85373, "part_value": 8647796.22d }
+{ "partkey": 122717, "part_value": 8646758.540000001d }
+{ "partkey": 113431, "part_value": 8646348.34d }
+{ "partkey": 66015, "part_value": 8643349.4d }
+{ "partkey": 33141, "part_value": 8643243.18d }
+{ "partkey": 69786, "part_value": 8637396.92d }
+{ "partkey": 181857, "part_value": 8637393.28d }
+{ "partkey": 122939, "part_value": 8636378.0d }
+{ "partkey": 196223, "part_value": 8635391.02d }
+{ "partkey": 50532, "part_value": 8632648.24d }
+{ "partkey": 58102, "part_value": 8632614.54d }
+{ "partkey": 93581, "part_value": 8632372.36d }
+{ "partkey": 52804, "part_value": 8632109.25d }
+{ "partkey": 755, "part_value": 8627091.68d }
+{ "partkey": 16597, "part_value": 8623357.05d }
+{ "partkey": 119041, "part_value": 8622397.0d }
+{ "partkey": 89050, "part_value": 8621185.98d }
+{ "partkey": 98696, "part_value": 8620784.82d }
+{ "partkey": 94399, "part_value": 8620524.0d }
+{ "partkey": 151295, "part_value": 8616671.02d }
+{ "partkey": 56417, "part_value": 8613450.35d }
+{ "partkey": 121322, "part_value": 8612948.23d }
+{ "partkey": 126883, "part_value": 8611373.42d }
+{ "partkey": 29155, "part_value": 8610163.639999999d }
+{ "partkey": 114530, "part_value": 8608471.74d }
+{ "partkey": 131007, "part_value": 8607394.82d }
+{ "partkey": 128715, "part_value": 8606833.620000001d }
+{ "partkey": 72522, "part_value": 8601479.98d }
+{ "partkey": 144061, "part_value": 8595718.74d }
+{ "partkey": 83503, "part_value": 8595034.2d }
+{ "partkey": 112199, "part_value": 8590717.44d }
+{ "partkey": 9227, "part_value": 8587350.42d }
+{ "partkey": 116318, "part_value": 8585910.66d }
+{ "partkey": 41248, "part_value": 8585559.639999999d }
+{ "partkey": 159398, "part_value": 8584821.0d }
+{ "partkey": 105966, "part_value": 8582308.790000001d }
+{ "partkey": 137876, "part_value": 8580641.3d }
+{ "partkey": 122272, "part_value": 8580400.77d }
+{ "partkey": 195717, "part_value": 8577278.1d }
+{ "partkey": 165295, "part_value": 8571121.92d }
+{ "partkey": 5840, "part_value": 8570728.74d }
+{ "partkey": 120860, "part_value": 8570610.440000001d }
+{ "partkey": 66692, "part_value": 8567540.52d }
+{ "partkey": 135596, "part_value": 8563276.309999999d }
+{ "partkey": 150576, "part_value": 8562794.1d }
+{ "partkey": 7500, "part_value": 8562393.84d }
+{ "partkey": 107716, "part_value": 8561541.56d }
+{ "partkey": 100611, "part_value": 8559995.85d }
+{ "partkey": 171192, "part_value": 8557390.08d }
+{ "partkey": 107660, "part_value": 8556696.6d }
+{ "partkey": 13461, "part_value": 8556545.120000001d }
+{ "partkey": 90310, "part_value": 8555131.51d }
+{ "partkey": 141493, "part_value": 8553782.93d }
+{ "partkey": 71286, "part_value": 8552682.0d }
+{ "partkey": 136423, "part_value": 8551300.76d }
+{ "partkey": 54241, "part_value": 8550785.25d }
+{ "partkey": 120325, "part_value": 8549976.6d }
+{ "partkey": 424, "part_value": 8547527.1d }
+{ "partkey": 196543, "part_value": 8545907.09d }
+{ "partkey": 13042, "part_value": 8542717.18d }
+{ "partkey": 58332, "part_value": 8536074.69d }
+{ "partkey": 9191, "part_value": 8535663.92d }
+{ "partkey": 134357, "part_value": 8535429.9d }
+{ "partkey": 96207, "part_value": 8534900.6d }
+{ "partkey": 92292, "part_value": 8530618.78d }
+{ "partkey": 181093, "part_value": 8528303.52d }
+{ "partkey": 105064, "part_value": 8527491.6d }
+{ "partkey": 59635, "part_value": 8526854.08d }
+{ "partkey": 136974, "part_value": 8524351.56d }
+{ "partkey": 126694, "part_value": 8522783.37d }
+{ "partkey": 6247, "part_value": 8522606.9d }
+{ "partkey": 139447, "part_value": 8522521.92d }
+{ "partkey": 96313, "part_value": 8520949.92d }
+{ "partkey": 108454, "part_value": 8520916.25d }
+{ "partkey": 181254, "part_value": 8519496.1d }
+{ "partkey": 71117, "part_value": 8519223.0d }
+{ "partkey": 131703, "part_value": 8517215.280000001d }
+{ "partkey": 59312, "part_value": 8510568.36d }
+{ "partkey": 2903, "part_value": 8509960.35d }
+{ "partkey": 102838, "part_value": 8509527.69d }
+{ "partkey": 162806, "part_value": 8508906.05d }
+{ "partkey": 41527, "part_value": 8508222.36d }
+{ "partkey": 118416, "part_value": 8505858.36d }
+{ "partkey": 180203, "part_value": 8505024.16d }
+{ "partkey": 14773, "part_value": 8500598.28d }
+{ "partkey": 140446, "part_value": 8499514.24d }
+{ "partkey": 199641, "part_value": 8497362.59d }
+{ "partkey": 109240, "part_value": 8494617.12d }
+{ "partkey": 150268, "part_value": 8494188.379999999d }
+{ "partkey": 45310, "part_value": 8492380.65d }
+{ "partkey": 36552, "part_value": 8490733.6d }
+{ "partkey": 199690, "part_value": 8490145.8d }
+{ "partkey": 185353, "part_value": 8488726.68d }
+{ "partkey": 163615, "part_value": 8484985.01d }
+{ "partkey": 196520, "part_value": 8483545.040000001d }
+{ "partkey": 133438, "part_value": 8483482.35d }
+{ "partkey": 77285, "part_value": 8481442.32d }
+{ "partkey": 55824, "part_value": 8476893.9d }
+{ "partkey": 76753, "part_value": 8475522.12d }
+{ "partkey": 46129, "part_value": 8472717.959999999d }
+{ "partkey": 28358, "part_value": 8472515.5d }
+{ "partkey": 9317, "part_value": 8472145.32d }
+{ "partkey": 33823, "part_value": 8469721.44d }
+{ "partkey": 39055, "part_value": 8469145.07d }
+{ "partkey": 91471, "part_value": 8468874.56d }
+{ "partkey": 142299, "part_value": 8466039.549999999d }
+{ "partkey": 97672, "part_value": 8464119.8d }
+{ "partkey": 134712, "part_value": 8461781.790000001d }
+{ "partkey": 157988, "part_value": 8460123.200000001d }
+{ "partkey": 102284, "part_value": 8458652.44d }
+{ "partkey": 73533, "part_value": 8458453.32d }
+{ "partkey": 90599, "part_value": 8457874.86d }
+{ "partkey": 112160, "part_value": 8457863.360000001d }
+{ "partkey": 124792, "part_value": 8457633.7d }
+{ "partkey": 66097, "part_value": 8457573.15d }
+{ "partkey": 165271, "part_value": 8456969.01d }
+{ "partkey": 146925, "part_value": 8454887.91d }
+{ "partkey": 164277, "part_value": 8454838.5d }
+{ "partkey": 131290, "part_value": 8454811.2d }
+{ "partkey": 179386, "part_value": 8450909.899999999d }
+{ "partkey": 90486, "part_value": 8447873.86d }
+{ "partkey": 175924, "part_value": 8444421.66d }
+{ "partkey": 185922, "part_value": 8442394.88d }
+{ "partkey": 38492, "part_value": 8436438.32d }
+{ "partkey": 172511, "part_value": 8436287.34d }
+{ "partkey": 139539, "part_value": 8434180.29d }
+{ "partkey": 11926, "part_value": 8433199.52d }
+{ "partkey": 55889, "part_value": 8431449.879999999d }
+{ "partkey": 163068, "part_value": 8431116.4d }
+{ "partkey": 138772, "part_value": 8428406.36d }
+{ "partkey": 126821, "part_value": 8425180.68d }
+{ "partkey": 22091, "part_value": 8420687.88d }
+{ "partkey": 55981, "part_value": 8419434.38d }
+{ "partkey": 100960, "part_value": 8419403.46d }
+{ "partkey": 172568, "part_value": 8417955.21d }
+{ "partkey": 63135, "part_value": 8415945.530000001d }
+{ "partkey": 137651, "part_value": 8413170.35d }
+{ "partkey": 191353, "part_value": 8413039.84d }
+{ "partkey": 62988, "part_value": 8411571.48d }
+{ "partkey": 103417, "part_value": 8411541.12d }
+{ "partkey": 12052, "part_value": 8411519.28d }
+{ "partkey": 104260, "part_value": 8408516.55d }
+{ "partkey": 157129, "part_value": 8405730.08d }
+{ "partkey": 77254, "part_value": 8405537.22d }
+{ "partkey": 112966, "part_value": 8403512.89d }
+{ "partkey": 168114, "part_value": 8402764.56d }
+{ "partkey": 49940, "part_value": 8402328.2d }
+{ "partkey": 52017, "part_value": 8398753.6d }
+{ "partkey": 176179, "part_value": 8398087.0d }
+{ "partkey": 100215, "part_value": 8395906.61d }
+{ "partkey": 61256, "part_value": 8392811.200000001d }
+{ "partkey": 15366, "part_value": 8388907.8d }
+{ "partkey": 109479, "part_value": 8388027.2d }
+{ "partkey": 66202, "part_value": 8386522.83d }
+{ "partkey": 81707, "part_value": 8385761.1899999995d }
+{ "partkey": 51727, "part_value": 8385426.4d }
+{ "partkey": 9980, "part_value": 8382754.62d }
+{ "partkey": 174403, "part_value": 8378575.73d }
+{ "partkey": 54558, "part_value": 8378041.92d }
+{ "partkey": 3141, "part_value": 8377378.22d }
+{ "partkey": 134829, "part_value": 8377105.52d }
+{ "partkey": 145056, "part_value": 8376920.760000001d }
+{ "partkey": 194020, "part_value": 8375157.640000001d }
+{ "partkey": 7117, "part_value": 8373982.27d }
+{ "partkey": 120146, "part_value": 8373796.2d }
+{ "partkey": 126843, "part_value": 8370761.28d }
+{ "partkey": 62117, "part_value": 8369493.4399999995d }
+{ "partkey": 111221, "part_value": 8367525.81d }
+{ "partkey": 159337, "part_value": 8366092.260000001d }
+{ "partkey": 173903, "part_value": 8365428.4799999995d }
+{ "partkey": 136438, "part_value": 8364065.45d }
+{ "partkey": 56684, "part_value": 8363198.0d }
+{ "partkey": 137597, "part_value": 8363185.94d }
+{ "partkey": 20039, "part_value": 8361138.24d }
+{ "partkey": 121326, "part_value": 8359635.52d }
+{ "partkey": 48435, "part_value": 8352863.1d }
+{ "partkey": 1712, "part_value": 8349107.0d }
+{ "partkey": 167190, "part_value": 8347238.699999999d }
+{ "partkey": 32113, "part_value": 8346452.04d }
+{ "partkey": 40580, "part_value": 8342983.32d }
+{ "partkey": 74785, "part_value": 8342519.13d }
+{ "partkey": 14799, "part_value": 8342236.75d }
+{ "partkey": 177291, "part_value": 8341736.83d }
+{ "partkey": 198956, "part_value": 8340370.649999999d }
+{ "partkey": 69179, "part_value": 8338465.99d }
+{ "partkey": 118764, "part_value": 8337616.56d }
+{ "partkey": 128814, "part_value": 8336435.56d }
+{ "partkey": 82729, "part_value": 8331766.879999999d }
+{ "partkey": 152048, "part_value": 8330638.99d }
+{ "partkey": 171085, "part_value": 8326259.499999999d }
+{ "partkey": 126730, "part_value": 8325974.399999999d }
+{ "partkey": 77525, "part_value": 8323282.5d }
+{ "partkey": 170653, "part_value": 8322840.5d }
+{ "partkey": 5257, "part_value": 8320350.779999999d }
+{ "partkey": 67350, "part_value": 8318987.56d }
+{ "partkey": 109008, "part_value": 8317836.54d }
+{ "partkey": 199043, "part_value": 8316603.539999999d }
+{ "partkey": 139969, "part_value": 8316551.54d }
+{ "partkey": 22634, "part_value": 8316531.24d }
+{ "partkey": 173309, "part_value": 8315750.25d }
+{ "partkey": 10887, "part_value": 8315019.36d }
+{ "partkey": 42392, "part_value": 8312895.96d }
+{ "partkey": 126040, "part_value": 8312623.2d }
+{ "partkey": 101590, "part_value": 8304555.420000001d }
+{ "partkey": 46891, "part_value": 8302192.12d }
+{ "partkey": 138721, "part_value": 8301745.62d }
+{ "partkey": 113715, "part_value": 8301533.199999999d }
+{ "partkey": 78778, "part_value": 8299685.64d }
+{ "partkey": 142908, "part_value": 8299447.77d }
+{ "partkey": 64419, "part_value": 8297631.8d }
+{ "partkey": 21396, "part_value": 8296272.27d }
+{ "partkey": 4180, "part_value": 8295646.92d }
+{ "partkey": 63534, "part_value": 8295383.67d }
+{ "partkey": 135957, "part_value": 8294389.859999999d }
+{ "partkey": 30126, "part_value": 8291920.32d }
+{ "partkey": 158427, "part_value": 8288938.0d }
+{ "partkey": 14545, "part_value": 8288395.92d }
+{ "partkey": 75548, "part_value": 8288287.199999999d }
+{ "partkey": 64473, "part_value": 8286137.4399999995d }
+{ "partkey": 149553, "part_value": 8285714.88d }
+{ "partkey": 151284, "part_value": 8283526.649999999d }
+{ "partkey": 171091, "part_value": 8282934.36d }
+{ "partkey": 194256, "part_value": 8278985.34d }
+{ "partkey": 952, "part_value": 8276136.0d }
+{ "partkey": 121541, "part_value": 8275390.260000001d }
+{ "partkey": 177664, "part_value": 8275315.200000001d }
+{ "partkey": 51117, "part_value": 8274504.300000001d }
+{ "partkey": 66770, "part_value": 8273407.8d }
+{ "partkey": 37238, "part_value": 8272728.06d }
+{ "partkey": 46679, "part_value": 8270486.55d }
+{ "partkey": 165852, "part_value": 8268312.600000001d }
+{ "partkey": 99458, "part_value": 8266564.47d }
+{ "partkey": 114519, "part_value": 8265493.54d }
+{ "partkey": 7231, "part_value": 8264881.5d }
+{ "partkey": 19033, "part_value": 8264826.56d }
+{ "partkey": 125123, "part_value": 8262732.65d }
+{ "partkey": 18642, "part_value": 8261578.99d }
+{ "partkey": 50386, "part_value": 8261380.05d }
+{ "partkey": 193770, "part_value": 8259578.819999999d }
+{ "partkey": 7276, "part_value": 8258101.6d }
+{ "partkey": 178045, "part_value": 8253904.149999999d }
+{ "partkey": 49033, "part_value": 8253696.2299999995d }
+{ "partkey": 187195, "part_value": 8251334.58d }
+{ "partkey": 10590, "part_value": 8249227.4d }
+{ "partkey": 143779, "part_value": 8247057.7d }
+{ "partkey": 35205, "part_value": 8245675.17d }
+{ "partkey": 19729, "part_value": 8245081.600000001d }
+{ "partkey": 144946, "part_value": 8240479.8d }
+{ "partkey": 123786, "part_value": 8239581.239999999d }
+{ "partkey": 70843, "part_value": 8237973.2d }
+{ "partkey": 112437, "part_value": 8236907.5200000005d }
+{ "partkey": 5436, "part_value": 8236039.57d }
+{ "partkey": 163754, "part_value": 8235471.16d }
+{ "partkey": 115945, "part_value": 8234811.36d }
+{ "partkey": 27918, "part_value": 8233957.88d }
+{ "partkey": 105712, "part_value": 8233571.859999999d }
+{ "partkey": 41007, "part_value": 8229431.79d }
+{ "partkey": 40476, "part_value": 8226640.41d }
+{ "partkey": 145620, "part_value": 8221371.6d }
+{ "partkey": 7771, "part_value": 8220413.33d }
+{ "partkey": 86424, "part_value": 8215572.61d }
+{ "partkey": 129137, "part_value": 8215478.399999999d }
+{ "partkey": 76020, "part_value": 8210495.359999999d }
+{ "partkey": 140213, "part_value": 8209831.800000001d }
+{ "partkey": 32379, "part_value": 8208338.88d }
+{ "partkey": 130616, "part_value": 8207715.75d }
+{ "partkey": 195469, "part_value": 8206609.800000001d }
+{ "partkey": 191805, "part_value": 8205147.75d }
+{ "partkey": 90906, "part_value": 8200951.199999999d }
+{ "partkey": 170910, "part_value": 8195558.01d }
+{ "partkey": 105399, "part_value": 8193122.63d }
+{ "partkey": 123798, "part_value": 8192385.97d }
+{ "partkey": 90218, "part_value": 8191689.16d }
+{ "partkey": 114766, "part_value": 8189339.54d }
+{ "partkey": 11289, "part_value": 8187354.72d }
+{ "partkey": 178308, "part_value": 8185750.5d }
+{ "partkey": 71271, "part_value": 8185519.239999999d }
+{ "partkey": 1115, "part_value": 8184903.380000001d }
+{ "partkey": 152636, "part_value": 8184530.720000001d }
+{ "partkey": 151619, "part_value": 8182909.05d }
+{ "partkey": 116943, "part_value": 8181072.69d }
+{ "partkey": 28891, "part_value": 8181051.54d }
+{ "partkey": 47049, "part_value": 8180955.0d }
+{ "partkey": 158827, "part_value": 8180470.899999999d }
+{ "partkey": 92620, "part_value": 8179671.550000001d }
+{ "partkey": 20814, "part_value": 8176953.54d }
+{ "partkey": 179323, "part_value": 8176795.55d }
+{ "partkey": 193453, "part_value": 8174343.94d }
+{ "partkey": 56888, "part_value": 8173342.0d }
+{ "partkey": 28087, "part_value": 8169876.3d }
+{ "partkey": 164254, "part_value": 8169632.35d }
+{ "partkey": 57661, "part_value": 8168848.16d }
+{ "partkey": 7363, "part_value": 8167538.05d }
+{ "partkey": 164499, "part_value": 8167512.08d }
+{ "partkey": 197557, "part_value": 8165940.45d }
+{ "partkey": 5495, "part_value": 8164805.22d }
+{ "partkey": 966, "part_value": 8163824.79d }
+{ "partkey": 98435, "part_value": 8161771.45d }
+{ "partkey": 127227, "part_value": 8161344.92d }
+{ "partkey": 194100, "part_value": 8160978.78d }
+{ "partkey": 40134, "part_value": 8160358.08d }
+{ "partkey": 107341, "part_value": 8159952.05d }
+{ "partkey": 6790, "part_value": 8158792.66d }
+{ "partkey": 43851, "part_value": 8157101.4d }
+{ "partkey": 51295, "part_value": 8156419.2d }
+{ "partkey": 69512, "part_value": 8151537.0d }
+{ "partkey": 164274, "part_value": 8149869.93d }
+{ "partkey": 130854, "part_value": 8145338.85d }
+{ "partkey": 186865, "part_value": 8143586.82d }
+{ "partkey": 176629, "part_value": 8141411.2d }
+{ "partkey": 193739, "part_value": 8141377.7700000005d }
+{ "partkey": 6810, "part_value": 8139822.6d }
+{ "partkey": 27732, "part_value": 8136724.96d }
+{ "partkey": 50616, "part_value": 8134089.82d }
+{ "partkey": 123908, "part_value": 8128920.54d }
+{ "partkey": 140994, "part_value": 8128470.819999999d }
+{ "partkey": 99039, "part_value": 8128290.78d }
+{ "partkey": 62735, "part_value": 8124940.5d }
+{ "partkey": 47829, "part_value": 8122796.5d }
+{ "partkey": 192635, "part_value": 8122687.569999999d }
+{ "partkey": 192429, "part_value": 8119268.0d }
+{ "partkey": 145812, "part_value": 8119165.63d }
+{ "partkey": 42896, "part_value": 8118529.800000001d }
+{ "partkey": 146877, "part_value": 8118266.16d }
+{ "partkey": 60882, "part_value": 8116095.04d }
+{ "partkey": 18254, "part_value": 8114783.039999999d }
+{ "partkey": 165464, "part_value": 8114571.8d }
+{ "partkey": 57936, "part_value": 8111927.25d }
+{ "partkey": 52226, "part_value": 8110723.32d }
+{ "partkey": 128571, "part_value": 8106788.8d }
+{ "partkey": 100308, "part_value": 8105837.04d }
+{ "partkey": 8872, "part_value": 8102395.619999999d }
+{ "partkey": 58867, "part_value": 8102033.19d }
+{ "partkey": 145153, "part_value": 8100222.84d }
+{ "partkey": 172088, "part_value": 8098138.2d }
+{ "partkey": 59398, "part_value": 8095845.45d }
+{ "partkey": 89395, "part_value": 8093576.100000001d }
+{ "partkey": 171961, "part_value": 8093538.0d }
+{ "partkey": 88736, "part_value": 8090762.16d }
+{ "partkey": 174053, "part_value": 8090350.109999999d }
+{ "partkey": 102237, "part_value": 8089103.22d }
+{ "partkey": 43041, "part_value": 8086537.899999999d }
+{ "partkey": 110219, "part_value": 8085296.899999999d }
+{ "partkey": 126738, "part_value": 8084199.199999999d }
+{ "partkey": 44787, "part_value": 8083628.4d }
+{ "partkey": 31277, "part_value": 8083580.760000001d }
+{ "partkey": 93595, "part_value": 8082188.8d }
+{ "partkey": 189040, "part_value": 8080257.210000001d }
+{ "partkey": 59851, "part_value": 8079024.239999999d }
+{ "partkey": 175100, "part_value": 8077904.01d }
+{ "partkey": 43429, "part_value": 8076729.96d }
+{ "partkey": 154199, "part_value": 8074940.76d }
+{ "partkey": 60963, "part_value": 8073894.4d }
+{ "partkey": 8768, "part_value": 8072760.96d }
+{ "partkey": 66095, "part_value": 8071421.7d }
+{ "partkey": 111552, "part_value": 8068184.48d }
+{ "partkey": 24563, "part_value": 8067500.399999999d }
+{ "partkey": 16167, "part_value": 8067495.239999999d }
+{ "partkey": 12662, "part_value": 8067248.850000001d }
+{ "partkey": 94540, "part_value": 8063727.16d }
+{ "partkey": 23308, "part_value": 8063463.180000001d }
+{ "partkey": 27390, "part_value": 8062823.25d }
+{ "partkey": 130660, "part_value": 8062787.48d }
+{ "partkey": 8608, "part_value": 8062411.16d }
+{ "partkey": 181552, "part_value": 8062008.3d }
+{ "partkey": 199319, "part_value": 8060248.56d }
+{ "partkey": 55475, "part_value": 8058850.92d }
+{ "partkey": 142711, "part_value": 8057926.58d }
+{ "partkey": 103499, "part_value": 8056978.0d }
+{ "partkey": 105943, "part_value": 8056698.75d }
+{ "partkey": 8432, "part_value": 8053052.16d }
+{ "partkey": 149392, "part_value": 8049675.6899999995d }
+{ "partkey": 101248, "part_value": 8048855.49d }
+{ "partkey": 140962, "part_value": 8047260.7d }
+{ "partkey": 87101, "part_value": 8046651.83d }
+{ "partkey": 133107, "part_value": 8046476.73d }
+{ "partkey": 45126, "part_value": 8045924.399999999d }
+{ "partkey": 87508, "part_value": 8042966.39d }
+{ "partkey": 124711, "part_value": 8042722.72d }
+{ "partkey": 173169, "part_value": 8042224.41d }
+{ "partkey": 175161, "part_value": 8041331.9799999995d }
+{ "partkey": 167787, "part_value": 8040075.779999999d }
+{ "partkey": 3242, "part_value": 8038855.53d }
+{ "partkey": 114789, "part_value": 8038628.35d }
+{ "partkey": 43833, "part_value": 8038545.83d }
+{ "partkey": 141198, "part_value": 8035110.72d }
+{ "partkey": 137248, "part_value": 8034109.35d }
+{ "partkey": 96673, "part_value": 8033491.2d }
+{ "partkey": 32180, "part_value": 8032380.720000001d }
+{ "partkey": 166493, "part_value": 8031902.399999999d }
+{ "partkey": 66959, "part_value": 8031839.4d }
+{ "partkey": 85628, "part_value": 8029693.4399999995d }
+{ "partkey": 110971, "part_value": 8029469.7d }
+{ "partkey": 130395, "part_value": 8027463.920000001d }
+{ "partkey": 7757, "part_value": 8026840.37d }
+{ "partkey": 178446, "part_value": 8025379.09d }
+{ "partkey": 41295, "part_value": 8024785.53d }
+{ "partkey": 100956, "part_value": 8024179.300000001d }
+{ "partkey": 131917, "part_value": 8021604.78d }
+{ "partkey": 24224, "part_value": 8020463.52d }
+{ "partkey": 2073, "part_value": 8020009.64d }
+{ "partkey": 121622, "part_value": 8018462.17d }
+{ "partkey": 14357, "part_value": 8016906.3d }
+{ "partkey": 135601, "part_value": 8016209.4399999995d }
+{ "partkey": 58458, "part_value": 8016192.52d }
+{ "partkey": 73036, "part_value": 8015799.0d }
+{ "partkey": 184722, "part_value": 8015680.3100000005d }
+{ "partkey": 151664, "part_value": 8014821.96d }
+{ "partkey": 195090, "part_value": 8012680.199999999d }
+{ "partkey": 162609, "part_value": 8011241.0d }
+{ "partkey": 83532, "part_value": 8009753.850000001d }
+{ "partkey": 50166, "part_value": 8007137.89d }
+{ "partkey": 181562, "part_value": 8006805.960000001d }
+{ "partkey": 175165, "part_value": 8005319.76d }
+{ "partkey": 62500, "part_value": 8005316.279999999d }
+{ "partkey": 36342, "part_value": 8004333.4d }
+{ "partkey": 128435, "part_value": 8004242.88d }
+{ "partkey": 92516, "part_value": 8003836.800000001d }
+{ "partkey": 30802, "part_value": 8003710.88d }
+{ "partkey": 107418, "part_value": 8000430.3d }
+{ "partkey": 46620, "part_value": 7999778.350000001d }
+{ "partkey": 191803, "part_value": 7994734.15d }
+{ "partkey": 106343, "part_value": 7993087.76d }
+{ "partkey": 59362, "part_value": 7990397.46d }
+{ "partkey": 8329, "part_value": 7990052.899999999d }
+{ "partkey": 75133, "part_value": 7988244.0d }
+{ "partkey": 179023, "part_value": 7986829.62d }
+{ "partkey": 135899, "part_value": 7985726.64d }
+{ "partkey": 5824, "part_value": 7985340.02d }
+{ "partkey": 148579, "part_value": 7984889.56d }
+{ "partkey": 95888, "part_value": 7984735.720000001d }
+{ "partkey": 9791, "part_value": 7982699.790000001d }
+{ "partkey": 170437, "part_value": 7982370.720000001d }
+{ "partkey": 39782, "part_value": 7977858.239999999d }
+{ "partkey": 20605, "part_value": 7977556.0d }
+{ "partkey": 28682, "part_value": 7976959.999999999d }
+{ "partkey": 42172, "part_value": 7973399.0d }
+{ "partkey": 56137, "part_value": 7971405.4d }
+{ "partkey": 64729, "part_value": 7970769.720000001d }
+{ "partkey": 98643, "part_value": 7968603.7299999995d }
+{ "partkey": 153787, "part_value": 7967535.58d }
+{ "partkey": 8932, "part_value": 7967222.1899999995d }
+{ "partkey": 20134, "part_value": 7965713.28d }
+{ "partkey": 197635, "part_value": 7963507.58d }
+{ "partkey": 80408, "part_value": 7963312.17d }
+{ "partkey": 37728, "part_value": 7961875.68d }
+{ "partkey": 26624, "part_value": 7961772.3100000005d }
+{ "partkey": 44736, "part_value": 7961144.1d }
+{ "partkey": 29763, "part_value": 7960605.03d }
+{ "partkey": 36147, "part_value": 7959463.68d }
+{ "partkey": 146040, "part_value": 7957587.66d }
+{ "partkey": 115469, "part_value": 7957485.140000001d }
+{ "partkey": 142276, "part_value": 7956790.63d }
+{ "partkey": 181280, "part_value": 7954037.35d }
+{ "partkey": 115096, "part_value": 7953047.55d }
+{ "partkey": 109650, "part_value": 7952258.73d }
+{ "partkey": 93862, "part_value": 7951992.24d }
+{ "partkey": 158325, "part_value": 7950728.3d }
+{ "partkey": 55952, "part_value": 7950387.0600000005d }
+{ "partkey": 122397, "part_value": 7947106.2700000005d }
+{ "partkey": 28114, "part_value": 7946945.720000001d }
+{ "partkey": 11966, "part_value": 7945197.48d }
+{ "partkey": 47814, "part_value": 7944083.0d }
+{ "partkey": 85096, "part_value": 7943691.0600000005d }
+{ "partkey": 51657, "part_value": 7943593.7700000005d }
+{ "partkey": 196680, "part_value": 7943578.89d }
+{ "partkey": 13141, "part_value": 7942730.340000001d }
+{ "partkey": 193327, "part_value": 7941036.25d }
+{ "partkey": 152612, "part_value": 7940663.71d }
+{ "partkey": 139680, "part_value": 7939242.359999999d }
+{ "partkey": 31134, "part_value": 7938318.3d }
+{ "partkey": 45636, "part_value": 7937240.85d }
+{ "partkey": 56694, "part_value": 7936015.949999999d }
+{ "partkey": 8114, "part_value": 7933921.88d }
+{ "partkey": 71518, "part_value": 7932261.69d }
+{ "partkey": 72922, "part_value": 7930400.64d }
+{ "partkey": 146699, "part_value": 7929167.4d }
+{ "partkey": 92387, "part_value": 7928972.67d }
+{ "partkey": 186289, "part_value": 7928786.1899999995d }
+{ "partkey": 95952, "part_value": 7927972.78d }
+{ "partkey": 196514, "part_value": 7927180.7d }
+{ "partkey": 4403, "part_value": 7925729.040000001d }
+{ "partkey": 2267, "part_value": 7925649.37d }
+{ "partkey": 45924, "part_value": 7925047.68d }
+{ "partkey": 11493, "part_value": 7916722.23d }
+{ "partkey": 104478, "part_value": 7916253.600000001d }
+{ "partkey": 166794, "part_value": 7913842.0d }
+{ "partkey": 161995, "part_value": 7910874.27d }
+{ "partkey": 23538, "part_value": 7909752.0600000005d }
+{ "partkey": 41093, "part_value": 7909579.92d }
+{ "partkey": 112073, "part_value": 7908617.57d }
+{ "partkey": 92814, "part_value": 7908262.5d }
+{ "partkey": 88919, "part_value": 7907992.5d }
+{ "partkey": 79753, "part_value": 7907933.88d }
+{ "partkey": 108765, "part_value": 7905338.98d }
+{ "partkey": 146530, "part_value": 7905336.6d }
+{ "partkey": 71475, "part_value": 7903367.58d }
+{ "partkey": 36289, "part_value": 7901946.5d }
+{ "partkey": 61739, "part_value": 7900794.0d }
+{ "partkey": 52338, "part_value": 7898638.08d }
+{ "partkey": 194299, "part_value": 7898421.24d }
+{ "partkey": 105235, "part_value": 7897829.9399999995d }
+{ "partkey": 77207, "part_value": 7897752.72d }
+{ "partkey": 96712, "part_value": 7897575.2700000005d }
+{ "partkey": 10157, "part_value": 7897046.25d }
+{ "partkey": 171154, "part_value": 7896814.5d }
+{ "partkey": 79373, "part_value": 7896186.0d }
+{ "partkey": 113808, "part_value": 7893353.88d }
+{ "partkey": 27901, "part_value": 7892952.0d }
+{ "partkey": 128820, "part_value": 7892882.72d }
+{ "partkey": 25891, "part_value": 7890511.2d }
+{ "partkey": 122819, "part_value": 7888881.02d }
+{ "partkey": 154731, "part_value": 7888301.33d }
+{ "partkey": 101674, "part_value": 7879324.6d }
+{ "partkey": 51968, "part_value": 7879102.21d }
+{ "partkey": 72073, "part_value": 7877736.11d }
+{ "partkey": 5182, "part_value": 7874521.7299999995d }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q12_shipping.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q12_shipping.3.query.adm
new file mode 100644
index 0000000..fb5e49b
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q12_shipping.3.query.adm
@@ -0,0 +1,2 @@
+{ "l_shipmode": "MAIL", "high_line_count": 6202, "low_line_count": 9324 }
+{ "l_shipmode": "SHIP", "high_line_count": 6200, "low_line_count": 9262 }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q13_customer_distribution.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q13_customer_distribution.3.query.adm
new file mode 100644
index 0000000..cea1e98
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q13_customer_distribution.3.query.adm
@@ -0,0 +1,42 @@
+{ "c_count": 0i64, "custdist": 50005i64 }
+{ "c_count": 9i64, "custdist": 6641i64 }
+{ "c_count": 10i64, "custdist": 6532i64 }
+{ "c_count": 11i64, "custdist": 6014i64 }
+{ "c_count": 8i64, "custdist": 5937i64 }
+{ "c_count": 12i64, "custdist": 5639i64 }
+{ "c_count": 13i64, "custdist": 5024i64 }
+{ "c_count": 19i64, "custdist": 4793i64 }
+{ "c_count": 7i64, "custdist": 4687i64 }
+{ "c_count": 17i64, "custdist": 4587i64 }
+{ "c_count": 18i64, "custdist": 4529i64 }
+{ "c_count": 20i64, "custdist": 4516i64 }
+{ "c_count": 15i64, "custdist": 4505i64 }
+{ "c_count": 14i64, "custdist": 4446i64 }
+{ "c_count": 16i64, "custdist": 4273i64 }
+{ "c_count": 21i64, "custdist": 4190i64 }
+{ "c_count": 22i64, "custdist": 3623i64 }
+{ "c_count": 6i64, "custdist": 3265i64 }
+{ "c_count": 23i64, "custdist": 3225i64 }
+{ "c_count": 24i64, "custdist": 2742i64 }
+{ "c_count": 25i64, "custdist": 2086i64 }
+{ "c_count": 5i64, "custdist": 1948i64 }
+{ "c_count": 26i64, "custdist": 1612i64 }
+{ "c_count": 27i64, "custdist": 1179i64 }
+{ "c_count": 4i64, "custdist": 1007i64 }
+{ "c_count": 28i64, "custdist": 893i64 }
+{ "c_count": 29i64, "custdist": 593i64 }
+{ "c_count": 3i64, "custdist": 415i64 }
+{ "c_count": 30i64, "custdist": 376i64 }
+{ "c_count": 31i64, "custdist": 226i64 }
+{ "c_count": 32i64, "custdist": 148i64 }
+{ "c_count": 2i64, "custdist": 134i64 }
+{ "c_count": 33i64, "custdist": 75i64 }
+{ "c_count": 34i64, "custdist": 50i64 }
+{ "c_count": 35i64, "custdist": 37i64 }
+{ "c_count": 1i64, "custdist": 17i64 }
+{ "c_count": 36i64, "custdist": 14i64 }
+{ "c_count": 38i64, "custdist": 5i64 }
+{ "c_count": 37i64, "custdist": 5i64 }
+{ "c_count": 40i64, "custdist": 4i64 }
+{ "c_count": 41i64, "custdist": 2i64 }
+{ "c_count": 39i64, "custdist": 1i64 }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q15_top_supplier.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q15_top_supplier.3.query.adm
new file mode 100644
index 0000000..d4aa05f
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q15_top_supplier.3.query.adm
@@ -0,0 +1 @@
+{ "s_suppkey": 8449, "s_name": "Supplier#000008449", "s_address": "Wp34zim9qYFbVctdW", "s_phone": "20-469-856-8873", "total_revenue": 1772627.2086999998d }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q16_parts_supplier_relationship.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q16_parts_supplier_relationship.3.query.adm
new file mode 100644
index 0000000..9479929
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q16_parts_supplier_relationship.3.query.adm
@@ -0,0 +1,18314 @@
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 28i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 27i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 24i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 23i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 23i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 23i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 20i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 19i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 16i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 15i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 15i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 15i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 15i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 12i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 11i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 8i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED STEEL", "p_size": 45, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 7i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 6i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#12", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#13", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#14", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#15", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#21", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#23", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#24", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#25", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#31", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#33", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#34", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#35", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#41", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#42", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#43", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#44", "p_type": "STANDARD POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#51", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "ECONOMY POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "SMALL POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#52", "p_type": "STANDARD POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "LARGE POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#53", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "MEDIUM PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "SMALL POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#54", "p_type": "STANDARD POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "ECONOMY POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BRUSHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE BURNISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "LARGE POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "MEDIUM PLATED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO ANODIZED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED TIN", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO POLISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL ANODIZED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED BRASS", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL BURNISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL PLATED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED COPPER", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED COPPER", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED TIN", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "SMALL POLISHED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED STEEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD ANODIZED TIN", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED COPPER", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED COPPER", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED STEEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BRUSHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED BRASS", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED BRASS", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED STEEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED STEEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED STEEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD BURNISHED TIN", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED BRASS", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED BRASS", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED COPPER", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED NICKEL", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED STEEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED TIN", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED BRASS", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED BRASS", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED BRASS", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED COPPER", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED COPPER", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED COPPER", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED NICKEL", "p_size": 3, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED NICKEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED NICKEL", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED NICKEL", "p_size": 45, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED NICKEL", "p_size": 49, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED STEEL", "p_size": 14, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED STEEL", "p_size": 23, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED TIN", "p_size": 9, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED TIN", "p_size": 19, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD POLISHED TIN", "p_size": 36, "supplier_cnt": 4i64 }
+{ "p_brand": "Brand#11", "p_type": "SMALL BRUSHED TIN", "p_size": 19, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE PLATED NICKEL", "p_size": 45, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#15", "p_type": "LARGE POLISHED NICKEL", "p_size": 9, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#21", "p_type": "PROMO BURNISHED STEEL", "p_size": 45, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#22", "p_type": "STANDARD PLATED STEEL", "p_size": 23, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#25", "p_type": "LARGE PLATED STEEL", "p_size": 19, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#32", "p_type": "STANDARD ANODIZED COPPER", "p_size": 23, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#33", "p_type": "SMALL ANODIZED BRASS", "p_size": 9, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#35", "p_type": "MEDIUM ANODIZED TIN", "p_size": 19, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#51", "p_type": "SMALL PLATED BRASS", "p_size": 23, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#52", "p_type": "MEDIUM BRUSHED BRASS", "p_size": 45, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#53", "p_type": "MEDIUM BRUSHED TIN", "p_size": 45, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#54", "p_type": "ECONOMY POLISHED BRASS", "p_size": 9, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#55", "p_type": "PROMO PLATED BRASS", "p_size": 19, "supplier_cnt": 3i64 }
+{ "p_brand": "Brand#55", "p_type": "STANDARD PLATED TIN", "p_size": 49, "supplier_cnt": 3i64 }
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q17_small_quantity_order_revenue.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q17_small_quantity_order_revenue.3.query.adm
new file mode 100644
index 0000000..91a8da3
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q17_small_quantity_order_revenue.3.query.adm
@@ -0,0 +1 @@
+348406.0542857142d
\ No newline at end of file
diff --git a/asterix-benchmarks/src/main/resources/tpc-h/results/q20_potential_part_promotion.3.query.adm b/asterix-benchmarks/src/main/resources/tpc-h/results/q20_potential_part_promotion.3.query.adm
new file mode 100644
index 0000000..d10c192
--- /dev/null
+++ b/asterix-benchmarks/src/main/resources/tpc-h/results/q20_potential_part_promotion.3.query.adm
@@ -0,0 +1,186 @@
+{ "s_name": "Supplier#000000020", "s_address": "iybAE,RmTymrZVYaFZva2SH,j" }
+{ "s_name": "Supplier#000000091", "s_address": "YV45D7TkfdQanOOZ7q9QxkyGUapU1oOWU6q3" }
+{ "s_name": "Supplier#000000205", "s_address": "rF uV8d0JNEk" }
+{ "s_name": "Supplier#000000285", "s_address": "Br7e1nnt1yxrw6ImgpJ7YdhFDjuBf" }
+{ "s_name": "Supplier#000000287", "s_address": "7a9SP7qW5Yku5PvSg" }
+{ "s_name": "Supplier#000000354", "s_address": "w8fOo5W,aS" }
+{ "s_name": "Supplier#000000378", "s_address": "FfbhyCxWvcPrO8ltp9" }
+{ "s_name": "Supplier#000000402", "s_address": "i9Sw4DoyMhzhKXCH9By,AYSgmD" }
+{ "s_name": "Supplier#000000530", "s_address": "0qwCMwobKY OcmLyfRXlagA8ukENJv," }
+{ "s_name": "Supplier#000000555", "s_address": "TfB,a5bfl3Ah 3Z 74GqnNs6zKVGM" }
+{ "s_name": "Supplier#000000640", "s_address": "mvvtlQKsTOsJj5Ihk7,cq" }
+{ "s_name": "Supplier#000000729", "s_address": "pqck2ppy758TQpZCUAjPvlU55K3QjfL7Bi" }
+{ "s_name": "Supplier#000000736", "s_address": "l6i2nMwVuovfKnuVgaSGK2rDy65DlAFLegiL7" }
+{ "s_name": "Supplier#000000761", "s_address": "zlSLelQUj2XrvTTFnv7WAcYZGvvMTx882d4" }
+{ "s_name": "Supplier#000000887", "s_address": "urEaTejH5POADP2ARrf" }
+{ "s_name": "Supplier#000000935", "s_address": "ij98czM 2KzWe7dDTOxB8sq0UfCdvrX" }
+{ "s_name": "Supplier#000000975", "s_address": ",AC e,tBpNwKb5xMUzeohxlRn, hdZJo73gFQF8y" }
+{ "s_name": "Supplier#000001263", "s_address": "rQWr6nf8ZhB2TAiIDIvo5Io" }
+{ "s_name": "Supplier#000001367", "s_address": "42YSkFcAXMMcucsqeEefOE4HeCC" }
+{ "s_name": "Supplier#000001426", "s_address": "bPOCc086oFm8sLtS,fGrH" }
+{ "s_name": "Supplier#000001446", "s_address": "lch9HMNU1R7a0LIybsUodVknk6" }
+{ "s_name": "Supplier#000001500", "s_address": "wDmF5xLxtQch9ctVu," }
+{ "s_name": "Supplier#000001602", "s_address": "uKNWIeafaM644" }
+{ "s_name": "Supplier#000001626", "s_address": "UhxNRzUu1dtFmp0" }
+{ "s_name": "Supplier#000001682", "s_address": "pXTkGxrTQVyH1Rr" }
+{ "s_name": "Supplier#000001700", "s_address": "7hMlCof1Y5zLFg" }
+{ "s_name": "Supplier#000001726", "s_address": "TeRY7TtTH24sEword7yAaSkjx8" }
+{ "s_name": "Supplier#000001730", "s_address": "Rc8e,1Pybn r6zo0VJIEiD0UD vhk" }
+{ "s_name": "Supplier#000001746", "s_address": "qWsendlOekQG1aW4uq06uQaCm51se8lirv7 hBRd" }
+{ "s_name": "Supplier#000001806", "s_address": "M934fuZSnLW" }
+{ "s_name": "Supplier#000001855", "s_address": "MWk6EAeozXb" }
+{ "s_name": "Supplier#000001931", "s_address": "FpJbMU2h6ZR2eBv8I9NIxF" }
+{ "s_name": "Supplier#000002022", "s_address": " dwebGX7Id2pc25YvY33" }
+{ "s_name": "Supplier#000002036", "s_address": "20ytTtVObjKUUI2WCB0A" }
+{ "s_name": "Supplier#000002096", "s_address": "kuxseyLtq QPLXxm9ZUrnB6Kkh92JtK5cQzzXNU " }
+{ "s_name": "Supplier#000002117", "s_address": "MRtkgKolHJ9Wh X9J,urANHKDzvjr" }
+{ "s_name": "Supplier#000002204", "s_address": "uYmlr46C06udCqanj0KiRsoTQakZsEyssL" }
+{ "s_name": "Supplier#000002218", "s_address": "nODZw5q4dx kp0K5" }
+{ "s_name": "Supplier#000002243", "s_address": "nSOEV3JeOU79" }
+{ "s_name": "Supplier#000002245", "s_address": "hz2qWXWVjOyKhqPYMoEwz6zFkrTaDM" }
+{ "s_name": "Supplier#000002282", "s_address": "ES21K9dxoW1I1TzWCj7ekdlNwSWnv1Z 6mQ,BKn" }
+{ "s_name": "Supplier#000002303", "s_address": "nCoWfpB6YOymbgOht7ltfklpkHl" }
+{ "s_name": "Supplier#000002331", "s_address": "WRh2w5WFvRg7Z0S1AvSvHCL" }
+{ "s_name": "Supplier#000002373", "s_address": "RzHSxOTQmElCjxIBiVA52Z JB58rJhPRylR" }
+{ "s_name": "Supplier#000002419", "s_address": "qydBQd14I5l5mVXa4fYY" }
+{ "s_name": "Supplier#000002571", "s_address": "JZUugz04c iJFLrlGsz9O N,W 1rVHNIReyq" }
+{ "s_name": "Supplier#000002585", "s_address": "CsPoKpw2QuTY4AV1NkWuttneIa4SN" }
+{ "s_name": "Supplier#000002629", "s_address": "0Bw,q5Zp8su9XrzoCngZ3cAEXZwZ" }
+{ "s_name": "Supplier#000002721", "s_address": "HVdFAN2JHMQSpKm" }
+{ "s_name": "Supplier#000002730", "s_address": "lIFxR4fzm31C6,muzJwl84z" }
+{ "s_name": "Supplier#000002775", "s_address": "yDclaDaBD4ihH" }
+{ "s_name": "Supplier#000002799", "s_address": "lwr, 6L3gdfc79PQut,4XO6nQsTJY63cAyYO" }
+{ "s_name": "Supplier#000002934", "s_address": "m,trBENywSArwg3DhB" }
+{ "s_name": "Supplier#000002941", "s_address": "Naddba 8YTEKekZyP0" }
+{ "s_name": "Supplier#000003028", "s_address": "jouzgX0WZjhNMWLaH4fy" }
+{ "s_name": "Supplier#000003095", "s_address": "HxON3jJhUi3zjt,r mTD" }
+{ "s_name": "Supplier#000003143", "s_address": "hdolgh608uTkHh7t6qfSqkifKaiFjnCH" }
+{ "s_name": "Supplier#000003185", "s_address": "hMa535Cbf2mj1Nw4OWOKWVrsK0VdDkJURrdjSIJe" }
+{ "s_name": "Supplier#000003189", "s_address": "DWdPxt7 RnkZv6VOByR0em" }
+{ "s_name": "Supplier#000003201", "s_address": "E87yws6I,t0qNs4QW7UzExKiJnJDZWue" }
+{ "s_name": "Supplier#000003213", "s_address": "pxrRP4irQ1VoyfQ,dTf3" }
+{ "s_name": "Supplier#000003275", "s_address": "9xO4nyJ2QJcX6vGf" }
+{ "s_name": "Supplier#000003288", "s_address": "EDdfNt7E5Uc,xLTupoIgYL4yY7ujh," }
+{ "s_name": "Supplier#000003314", "s_address": "jnisU8MzqO4iUB3zsPcrysMw3DDUojS4q7LD" }
+{ "s_name": "Supplier#000003373", "s_address": "iy8VM48ynpc3N2OsBwAvhYakO2us9R1bi" }
+{ "s_name": "Supplier#000003421", "s_address": "Sh3dt9W5oeofFWovnFhrg," }
+{ "s_name": "Supplier#000003422", "s_address": "DJoCEapUeBXoV1iYiCcPFQvzsTv2ZI960" }
+{ "s_name": "Supplier#000003441", "s_address": "zvFJIzS,oUuShHjpcX" }
+{ "s_name": "Supplier#000003590", "s_address": "sy79CMLxqb,Cbo" }
+{ "s_name": "Supplier#000003607", "s_address": "lNqFHQYjwSAkf" }
+{ "s_name": "Supplier#000003625", "s_address": "qY588W0Yk5iaUy1RXTgNrEKrMAjBYHcKs" }
+{ "s_name": "Supplier#000003723", "s_address": "jZEp0OEythCLcS OmJSrFtxJ66bMlzSp" }
+{ "s_name": "Supplier#000003849", "s_address": "KgbZEaRk,6Q3mWvwh6uptrs1KRUHg 0" }
+{ "s_name": "Supplier#000003894", "s_address": "vvGC rameLOk" }
+{ "s_name": "Supplier#000003941", "s_address": "Pmb05mQfBMS618O7WKqZJ 9vyv" }
+{ "s_name": "Supplier#000004059", "s_address": "umEYZSq9RJ2WEzdsv9meU8rmqwzVLRgiZwC" }
+{ "s_name": "Supplier#000004207", "s_address": "tF64pwiOM4IkWjN3mS,e06WuAjLx" }
+{ "s_name": "Supplier#000004236", "s_address": "dl,HPtJmGipxYsSqn9wmqkuWjst,mCeJ8O6T" }
+{ "s_name": "Supplier#000004278", "s_address": "bBddbpBxIVp Di9" }
+{ "s_name": "Supplier#000004281", "s_address": "1OwPHh Pgiyeus,iZS5eA23JDOipwk" }
+{ "s_name": "Supplier#000004304", "s_address": "hQCAz59k,HLlp2CKUrcBIL" }
+{ "s_name": "Supplier#000004346", "s_address": "S3076LEOwo" }
+{ "s_name": "Supplier#000004406", "s_address": "Ah0ZaLu6VwufPWUz,7kbXgYZhauEaHqGIg" }
+{ "s_name": "Supplier#000004430", "s_address": "yvSsKNSTL5HLXBET4luOsPNLxKzAMk" }
+{ "s_name": "Supplier#000004527", "s_address": "p pVXCnxgcklWF6A1o3OHY3qW6" }
+{ "s_name": "Supplier#000004655", "s_address": "67NqBc4 t3PG3F8aO IsqWNq4kGaPowYL" }
+{ "s_name": "Supplier#000004851", "s_address": "Rj,x6IgLT7kBL99nqp" }
+{ "s_name": "Supplier#000004871", "s_address": ",phpt6AWEnUS8t4Avb50rFfdg7O9c6nU8xxv8eC5" }
+{ "s_name": "Supplier#000004884", "s_address": "42Z1uLye9nsn6aTGBNd dI8 x" }
+{ "s_name": "Supplier#000004975", "s_address": "GPq5PMKY6Wy" }
+{ "s_name": "Supplier#000005076", "s_address": "Xl7h9ifgvIHmqxFLgWfHK4Gjav BkP" }
+{ "s_name": "Supplier#000005195", "s_address": "Woi3b2ZaicPh ZSfu1EfXhE" }
+{ "s_name": "Supplier#000005256", "s_address": "Onc3t57VAMchm,pmoVLaU8bONni9NsuaM PzMMFz" }
+{ "s_name": "Supplier#000005257", "s_address": "f9g8SEHB7obMj3QXAjXS2vfYY22" }
+{ "s_name": "Supplier#000005300", "s_address": "gXG28YqpxU" }
+{ "s_name": "Supplier#000005323", "s_address": "tMCkdqbDoyNo8vMIkzjBqYexoRAuv,T6 qzcu" }
+{ "s_name": "Supplier#000005386", "s_address": "Ub6AAfHpWLWP" }
+{ "s_name": "Supplier#000005426", "s_address": "9Dz2OVT1q sb4BK71ljQ1XjPBYRPvO" }
+{ "s_name": "Supplier#000005465", "s_address": "63cYZenZBRZ613Q1FaoG0,smnC5zl9" }
+{ "s_name": "Supplier#000005484", "s_address": "saFdOR qW7AFY,3asPqiiAa11Mo22pCoN0BtPrKo" }
+{ "s_name": "Supplier#000005505", "s_address": "d2sbjG43KwMPX" }
+{ "s_name": "Supplier#000005506", "s_address": "On f5ypzoWgB" }
+{ "s_name": "Supplier#000005631", "s_address": "14TVrjlzo2SJEBYCDgpMwTlvwSqC" }
+{ "s_name": "Supplier#000005642", "s_address": "ZwKxAv3V40tW E8P7Qwu,zlu,kPsL" }
+{ "s_name": "Supplier#000005686", "s_address": "f2RBKec2T1NIi7yS M" }
+{ "s_name": "Supplier#000005730", "s_address": "5rkb0PSews HvxkL8JaD41UpnSF2cg8H1" }
+{ "s_name": "Supplier#000005736", "s_address": "2dq XTYhtYWSfp" }
+{ "s_name": "Supplier#000005737", "s_address": "dmEWcS32C3kx,d,B95 OmYn48" }
+{ "s_name": "Supplier#000005797", "s_address": ",o,OebwRbSDmVl9gN9fpWPCiqB UogvlSR" }
+{ "s_name": "Supplier#000005875", "s_address": "lK,sYiGzB94hSyHy9xvSZFbVQNCZe2LXZuGbS" }
+{ "s_name": "Supplier#000005974", "s_address": "REhR5jE,lLusQXvf54SwYySgsSSVFhu" }
+{ "s_name": "Supplier#000006059", "s_address": "4m0cv8MwJ9yX2vlwI Z" }
+{ "s_name": "Supplier#000006065", "s_address": "UiI2Cy3W4Tu5sLk LuvXLRy6KihlGv" }
+{ "s_name": "Supplier#000006093", "s_address": "KJNUg1odUT2wtCS2s6PrH3D6fd" }
+{ "s_name": "Supplier#000006099", "s_address": "aZilwQKYDTVPoK" }
+{ "s_name": "Supplier#000006109", "s_address": "rY5gbfh3dKHnylcQUTPGCwnbe" }
+{ "s_name": "Supplier#000006217", "s_address": "RVN23SYT9jenUeaWGXUd" }
+{ "s_name": "Supplier#000006297", "s_address": "73VRDOO56GUCyvc40oYJ" }
+{ "s_name": "Supplier#000006435", "s_address": "xIgE69XszYbnO4Eon7cHHO8y" }
+{ "s_name": "Supplier#000006463", "s_address": "7 wkdj2EO49iotley2kmIM ADpLSszGV3RNWj" }
+{ "s_name": "Supplier#000006478", "s_address": "bQYPnj9lpmW3U" }
+{ "s_name": "Supplier#000006521", "s_address": "b9 2zjHzxR" }
+{ "s_name": "Supplier#000006642", "s_address": "N,CUclSqRLJcS8zQ" }
+{ "s_name": "Supplier#000006659", "s_address": "iTLsnvD8D2GzWNUv kRInwRjk5rDeEmfup1" }
+{ "s_name": "Supplier#000006669", "s_address": "NQ4Yryj624p7K53" }
+{ "s_name": "Supplier#000006748", "s_address": "rC,2rEn8gKDIS5Q0dJEoiF" }
+{ "s_name": "Supplier#000006761", "s_address": "n4jhxGMqB5prD1HhpLvwrWStOLlla" }
+{ "s_name": "Supplier#000006808", "s_address": "HGd2Xo 9nEcHJhZvXjXxWKIpApT" }
+{ "s_name": "Supplier#000006858", "s_address": "fnlINT885vBBhsWwTGiZ0o22thwGY16h GHJj21" }
+{ "s_name": "Supplier#000006946", "s_address": "To6Slo0GJTqcIvD" }
+{ "s_name": "Supplier#000006949", "s_address": "mLxYUJhsGcLtKe ,GFirNu183AvT" }
+{ "s_name": "Supplier#000007072", "s_address": "2tRyX9M1a 4Rcm57s779F1ANG9jlpK" }
+{ "s_name": "Supplier#000007098", "s_address": "G3j8g0KC4OcbAu2OVoPHrXQWMCUdjq8wgCHOExu" }
+{ "s_name": "Supplier#000007132", "s_address": "xonvn0KAQIL3p8kYk HC1FSSDSUSTC" }
+{ "s_name": "Supplier#000007135", "s_address": "ls DoKV7V5ulfQy9V" }
+{ "s_name": "Supplier#000007147", "s_address": "Xzb16kC63wmLVYexUEgB0hXFvHkjT5iPpq" }
+{ "s_name": "Supplier#000007160", "s_address": "TqDGBULB3cTqIT6FKDvm9BS4e4v,zwYiQPb" }
+{ "s_name": "Supplier#000007169", "s_address": "tEc95D2moN9S84nd55O,dlnW" }
+{ "s_name": "Supplier#000007278", "s_address": "I2ae3rS7KVF8GVHtB" }
+{ "s_name": "Supplier#000007365", "s_address": "51xhROLvQMJ05DndtZWt" }
+{ "s_name": "Supplier#000007398", "s_address": "V8eE6oZ00OFNU," }
+{ "s_name": "Supplier#000007402", "s_address": "4UVv58ery1rjmqSR5" }
+{ "s_name": "Supplier#000007448", "s_address": "yhhpWiJi7EJ6Q5VCaQ" }
+{ "s_name": "Supplier#000007458", "s_address": "BYuucapYkptZl6fnd2QaDyZmI9gR1Ih16e" }
+{ "s_name": "Supplier#000007477", "s_address": "9m9j0wfhWzCvVHxkU,PpAxwSH0h" }
+{ "s_name": "Supplier#000007509", "s_address": "q8,V6LJRoHJjHcOuSG7aLTMg" }
+{ "s_name": "Supplier#000007561", "s_address": "rMcFg2530VC" }
+{ "s_name": "Supplier#000007616", "s_address": "R IovIqzDi3,QHnaqZk1xS4hGAgelhP4yj" }
+{ "s_name": "Supplier#000007760", "s_address": "JsPE18PvcdFTK" }
+{ "s_name": "Supplier#000007801", "s_address": "69fi,U1r6enUb " }
+{ "s_name": "Supplier#000007865", "s_address": "5cDGCS,T6N" }
+{ "s_name": "Supplier#000007885", "s_address": "u3sicchh5ZpyTUpN1cJKNcAoabIWgY" }
+{ "s_name": "Supplier#000007926", "s_address": "ErzCF80K9Uy" }
+{ "s_name": "Supplier#000007998", "s_address": "LnASFBfYRFOo9d6d,asBvVq9Lo2P" }
+{ "s_name": "Supplier#000008090", "s_address": "eonbJZvoDFYBNUinYfp6yERIg" }
+{ "s_name": "Supplier#000008224", "s_address": "TWxt9f,LVER" }
+{ "s_name": "Supplier#000008231", "s_address": "IK7eGw Yj90sTdpsP,vcqWxLB" }
+{ "s_name": "Supplier#000008243", "s_address": "2AyePMkDqmzVzjGTizXthFLo8h EiudCMxOmIIG" }
+{ "s_name": "Supplier#000008323", "s_address": "75I18sZmASwm POeheRMdj9tmpyeQ,BfCXN5BIAb" }
+{ "s_name": "Supplier#000008366", "s_address": "h778cEj14BuW9OEKlvPTWq4iwASR6EBBXN7zeS8" }
+{ "s_name": "Supplier#000008532", "s_address": "Uc29q4,5xVdDOF87UZrxhr4xWS0ihEUXuh" }
+{ "s_name": "Supplier#000008595", "s_address": "MH0iB73GQ3z UW3O DbCbqmc" }
+{ "s_name": "Supplier#000008610", "s_address": "SgVgP90vP452sUNTgzL9zKwXHXAzV6tV" }
+{ "s_name": "Supplier#000008683", "s_address": "gLuGcugfpJSeGQARnaHNCaWnGaqsNnjyl20" }
+{ "s_name": "Supplier#000008705", "s_address": "aE,trRNdPx,4yinTD9O3DebDIp" }
+{ "s_name": "Supplier#000008742", "s_address": "HmPlQEzKCPEcTUL14,kKq" }
+{ "s_name": "Supplier#000008841", "s_address": "I 85Lu1sekbg2xrSIzm0" }
+{ "s_name": "Supplier#000008872", "s_address": "8D 45GgxJO2OwwYP9S4AaXJKvDwPfLM" }
+{ "s_name": "Supplier#000008879", "s_address": "rDSA,D9oPM,65NMWEFrmGKAu" }
+{ "s_name": "Supplier#000008967", "s_address": "2kwEHyMG 7FwozNImAUE6mH0hYtqYculJM" }
+{ "s_name": "Supplier#000008972", "s_address": "w2vF6 D5YZO3visPXsqVfLADTK" }
+{ "s_name": "Supplier#000009032", "s_address": "qK,trB6Sdy4Dz1BRUFNy" }
+{ "s_name": "Supplier#000009043", "s_address": "57OPvKH4qyXIZ7IzYeCaw11a5N1Ki9f1WWmVQ," }
+{ "s_name": "Supplier#000009278", "s_address": "RqYTzgxj93CLX 0mcYfCENOefD" }
+{ "s_name": "Supplier#000009326", "s_address": "XmiC,uy36B9,fb0zhcjaagiXQutg" }
+{ "s_name": "Supplier#000009430", "s_address": "igRqmneFt " }
+{ "s_name": "Supplier#000009549", "s_address": "h3RVchUf8MzY46IzbZ0ng09" }
+{ "s_name": "Supplier#000009601", "s_address": "51m637bO,Rw5DnHWFUvLacRx9" }
+{ "s_name": "Supplier#000009709", "s_address": "rRnCbHYgDgl9PZYnyWKVYSUW0vKg" }
+{ "s_name": "Supplier#000009753", "s_address": "wLhVEcRmd7PkJF4FBnGK7Z" }
+{ "s_name": "Supplier#000009799", "s_address": " 4wNjXGa4OKWl" }
+{ "s_name": "Supplier#000009811", "s_address": "E3iuyq7UnZxU7oPZIe2Gu6" }
+{ "s_name": "Supplier#000009812", "s_address": "APFRMy3lCbgFga53n5t9DxzFPQPgnjrGt32" }
+{ "s_name": "Supplier#000009846", "s_address": "57sNwJJ3PtBDu,hMPP5QvpcOcSNRXn3PypJJrh" }
+{ "s_name": "Supplier#000009899", "s_address": "7XdpAHrzr1t,UQFZE" }
+{ "s_name": "Supplier#000009974", "s_address": "7wJ,J5DKcxSU4Kp1cQLpbcAvB5AsvKT" }
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
index 14975ff..edd4b2a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
@@ -14,12 +14,15 @@
*/
package edu.uci.ics.asterix.common.api;
+import java.util.concurrent.Callable;
import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
public class AsterixThreadExecutor implements Executor {
public final static AsterixThreadExecutor INSTANCE = new AsterixThreadExecutor();
- private final Executor executor = Executors.newCachedThreadPool(AsterixThreadFactory.INSTANCE);
+ private final ExecutorService executorService = Executors.newCachedThreadPool(AsterixThreadFactory.INSTANCE);
private AsterixThreadExecutor() {
@@ -27,6 +30,10 @@
@Override
public void execute(Runnable command) {
- executor.execute(command);
+ executorService.execute(command);
+ }
+
+ public Future<Object> submit(Callable command) {
+ return (Future<Object>) executorService.submit(command);
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
index d035303..4287212 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -15,6 +15,7 @@
package edu.uci.ics.asterix.common.api;
import java.io.IOException;
+import java.util.List;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -62,5 +63,5 @@
public double getBloomFilterFalsePositiveRate();
- public IVirtualBufferCache getVirtualBufferCache(int datasetID);
+ public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID);
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
index 85f90e4..d7ff15d 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
@@ -31,6 +31,9 @@
private static final String STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY = "storage.memorycomponent.numpages";
private static final int STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT = 1024; // ... so 32MB components
+ private static final String STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY = "storage.memorycomponent.numcomponents";
+ private static final int STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_DEFAULT = 2; // 2 components
+
private static final String STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY = "storage.memorycomponent.globalbudget";
private static final long STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT = 536870912; // 512MB
@@ -53,7 +56,7 @@
return accessor.getProperty(STORAGE_BUFFERCACHE_SIZE_KEY, STORAGE_BUFFERCACHE_SIZE_DEFAULT,
PropertyInterpreters.getLongPropertyInterpreter());
}
-
+
public int getBufferCacheNumPages() {
return (int) (getBufferCacheSize() / getBufferCachePageSize());
}
@@ -73,6 +76,11 @@
PropertyInterpreters.getIntegerPropertyInterpreter());
}
+ public int getMemoryComponentsNum() {
+ return accessor.getProperty(STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY,
+ STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_DEFAULT, PropertyInterpreters.getIntegerPropertyInterpreter());
+ }
+
public long getMemoryComponentGlobalBudget() {
return accessor.getProperty(STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY,
STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT, PropertyInterpreters.getLongPropertyInterpreter());
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
index 6094a27..978577a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
@@ -25,13 +25,7 @@
private static final int TXN_LOG_BUFFER_PAGESIZE_DEFAULT = (128 << 10); // 128KB
private static final String TXN_LOG_PARTITIONSIZE_KEY = "txn.log.partitionsize";
- private static final long TXN_LOG_PARTITIONSIZE_DEFAULT = (2 << 30); // 2GB
-
- private static final String TXN_LOG_DISKSECTORSIZE_KEY = "txn.log.disksectorsize";
- private static final int TXN_LOG_DISKSECTORSIZE_DEFAULT = 4096;
-
- private static final String TXN_LOG_GROUPCOMMITINTERVAL_KEY = "txn.log.groupcommitinterval";
- private static int TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT = 10; // 0.1ms
+ private static final long TXN_LOG_PARTITIONSIZE_DEFAULT = ((long)2 << 30); // 2GB
private static final String TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY = "txn.log.checkpoint.lsnthreshold";
private static final int TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT = (64 << 20); // 64M
@@ -81,16 +75,6 @@
PropertyInterpreters.getLongPropertyInterpreter());
}
- public int getLogDiskSectorSize() {
- return accessor.getProperty(TXN_LOG_DISKSECTORSIZE_KEY, TXN_LOG_DISKSECTORSIZE_DEFAULT,
- PropertyInterpreters.getIntegerPropertyInterpreter());
- }
-
- public int getGroupCommitInterval() {
- return accessor.getProperty(TXN_LOG_GROUPCOMMITINTERVAL_KEY, TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT,
- PropertyInterpreters.getIntegerPropertyInterpreter());
- }
-
public int getCheckpointLSNThreshold() {
return accessor.getProperty(TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY, TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT,
PropertyInterpreters.getIntegerPropertyInterpreter());
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
index bd2828d..9efc9fd 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.asterix.common.context;
+import java.util.List;
+
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
@@ -29,9 +31,9 @@
}
@Override
- public IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx) {
+ public List<IVirtualBufferCache> getVirtualBufferCaches(IHyracksTaskContext ctx) {
return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
- .getVirtualBufferCache(datasetID);
+ .getVirtualBufferCaches(datasetID);
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
index c48115c..7e7ffd9 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
@@ -17,57 +17,34 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
public class BaseOperationTracker implements ILSMOperationTracker {
- protected final ILSMIOOperationCallback ioOpCallback;
- protected long lastLSN;
- protected long firstLSN;
+ protected final DatasetLifecycleManager datasetLifecycleManager;
+ protected final int datasetID;
- public BaseOperationTracker(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
- this.ioOpCallback = ioOpCallbackFactory == null ? NoOpIOOperationCallback.INSTANCE : ioOpCallbackFactory
- .createIOOperationCallback(this);
- resetLSNs();
- }
-
- public ILSMIOOperationCallback getIOOperationCallback() {
- return ioOpCallback;
- }
-
- public long getLastLSN() {
- return lastLSN;
- }
-
- public long getFirstLSN() {
- return firstLSN;
- }
-
- public void updateLastLSN(long lastLSN) {
- if (firstLSN == -1) {
- firstLSN = lastLSN;
- }
- this.lastLSN = Math.max(this.lastLSN, lastLSN);
- }
-
- public void resetLSNs() {
- lastLSN = -1;
- firstLSN = -1;
+ public BaseOperationTracker(DatasetLifecycleManager datasetLifecycleManager, int datasetID) {
+ this.datasetLifecycleManager = datasetLifecycleManager;
+ this.datasetID = datasetID;
}
@Override
public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
+ if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ datasetLifecycleManager.declareActiveIOOperation(datasetID);
+ }
}
@Override
public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
+ if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ datasetLifecycleManager.undeclareActiveIOOperation(datasetID);
+ }
}
@Override
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
index 3ffa73d..dee652e 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
@@ -15,10 +15,13 @@
package edu.uci.ics.asterix.common.context;
+import java.util.List;
+
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -34,12 +37,13 @@
this.ctx = ctx;
}
- public void diskComponentAdded(final ILSMIndex index, int totalNumDiskComponents) throws HyracksDataException,
- IndexException {
- if (!ctx.isShuttingdown() && totalNumDiskComponents >= threshold) {
+ @Override
+ public void diskComponentAdded(final ILSMIndex index) throws HyracksDataException, IndexException {
+ List<ILSMComponent> immutableComponents = index.getImmutableComponents();
+ if (!ctx.isShuttingdown() && immutableComponents.size() >= threshold) {
ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
NoOpOperationCallback.INSTANCE);
- accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+ accessor.scheduleMerge(index.getIOOperationCallback());
}
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
index 7232f01..6335fb1 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
@@ -26,12 +26,13 @@
import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
@@ -42,7 +43,7 @@
public class DatasetLifecycleManager implements IIndexLifecycleManager, ILifeCycleComponent {
private final AsterixStorageProperties storageProperties;
- private final Map<Integer, MultitenantVirtualBufferCache> datasetVirtualBufferCaches;
+ private final Map<Integer, List<IVirtualBufferCache>> datasetVirtualBufferCaches;
private final Map<Integer, ILSMOperationTracker> datasetOpTrackers;
private final Map<Integer, DatasetInfo> datasetInfos;
private final ILocalResourceRepository resourceRepository;
@@ -53,7 +54,7 @@
ILocalResourceRepository resourceRepository) {
this.storageProperties = storageProperties;
this.resourceRepository = resourceRepository;
- datasetVirtualBufferCaches = new HashMap<Integer, MultitenantVirtualBufferCache>();
+ datasetVirtualBufferCaches = new HashMap<Integer, List<IVirtualBufferCache>>();
datasetOpTrackers = new HashMap<Integer, ILSMOperationTracker>();
datasetInfos = new HashMap<Integer, DatasetInfo>();
capacity = storageProperties.getMemoryComponentGlobalBudget();
@@ -98,29 +99,67 @@
public synchronized void unregister(long resourceID) throws HyracksDataException {
int did = getDIDfromRID(resourceID);
DatasetInfo dsInfo = datasetInfos.get(did);
- IndexInfo iInfo = dsInfo.indexes.remove(resourceID);
+ IndexInfo iInfo = dsInfo.indexes.get(resourceID);
+
if (dsInfo == null || iInfo == null) {
throw new HyracksDataException("Index with resource ID " + resourceID + " does not exist.");
}
- if (iInfo.referenceCount != 0) {
- dsInfo.indexes.put(resourceID, iInfo);
+ PrimaryIndexOperationTracker opTracker = (PrimaryIndexOperationTracker) datasetOpTrackers.get(dsInfo.datasetID);
+ if (iInfo.referenceCount != 0 || (opTracker != null && opTracker.getNumActiveOperations() != 0)) {
throw new HyracksDataException("Cannot remove index while it is open.");
}
+ // TODO: use fine-grained counters, one for each index instead of a single counter per dataset.
+
+ // First wait for any ongoing IO operations
+ while (dsInfo.numActiveIOOps > 0) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ // Flush and wait for it to finish, it is separated from the above wait so they don't deadlock each other.
+ // TODO: Find a better way to do this.
+ flushAndWaitForIO(dsInfo, iInfo);
+
if (iInfo.isOpen) {
- iInfo.index.deactivate(true);
+ iInfo.index.deactivate(false);
}
+ dsInfo.indexes.remove(resourceID);
if (dsInfo.referenceCount == 0 && dsInfo.isOpen && dsInfo.indexes.isEmpty()) {
- IVirtualBufferCache vbc = getVirtualBufferCache(did);
- assert vbc != null;
- used -= (vbc.getNumPages() * vbc.getPageSize());
+ List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(did);
+ assert vbcs != null;
+ for (IVirtualBufferCache vbc : vbcs) {
+ used -= (vbc.getNumPages() * vbc.getPageSize());
+ }
datasetInfos.remove(did);
+ datasetVirtualBufferCaches.remove(did);
+ datasetOpTrackers.remove(did);
}
}
+ public synchronized void declareActiveIOOperation(int datasetID) throws HyracksDataException {
+ DatasetInfo dsInfo = datasetInfos.get(datasetID);
+ if (dsInfo == null) {
+ throw new HyracksDataException("Failed to find a dataset with ID " + datasetID);
+ }
+ dsInfo.incrementActiveIOOps();
+ }
+
+ public synchronized void undeclareActiveIOOperation(int datasetID) throws HyracksDataException {
+ DatasetInfo dsInfo = datasetInfos.get(datasetID);
+ if (dsInfo == null) {
+ throw new HyracksDataException("Failed to find a dataset with ID " + datasetID);
+ }
+ dsInfo.decrementActiveIOOps();
+ notifyAll();
+ }
+
@Override
public synchronized void open(long resourceID) throws HyracksDataException {
int did = getDIDfromRID(resourceID);
@@ -137,9 +176,12 @@
}
if (!dsInfo.isOpen) {
- IVirtualBufferCache vbc = getVirtualBufferCache(did);
- assert vbc != null;
- long additionalSize = vbc.getNumPages() * vbc.getPageSize();
+ List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(did);
+ assert vbcs != null;
+ long additionalSize = 0;
+ for (IVirtualBufferCache vbc : vbcs) {
+ additionalSize += vbc.getNumPages() * vbc.getPageSize();
+ }
while (used + additionalSize > capacity) {
if (!evictCandidateDataset()) {
throw new HyracksDataException("Cannot activate index since memory budget would be exceeded.");
@@ -165,26 +207,63 @@
List<DatasetInfo> datasetInfosList = new ArrayList<DatasetInfo>(datasetInfos.values());
Collections.sort(datasetInfosList);
for (DatasetInfo dsInfo : datasetInfosList) {
- ILSMOperationTracker opTracker = datasetOpTrackers.get(dsInfo.datasetID);
- if (opTracker != null && ((PrimaryIndexOperationTracker) opTracker).getNumActiveOperations() == 0
- && dsInfo.referenceCount == 0 && dsInfo.isOpen) {
+ PrimaryIndexOperationTracker opTracker = (PrimaryIndexOperationTracker) datasetOpTrackers
+ .get(dsInfo.datasetID);
+ if (opTracker != null && opTracker.getNumActiveOperations() == 0 && dsInfo.referenceCount == 0
+ && dsInfo.isOpen) {
+
+ // First wait for any ongoing IO operations
+ while (dsInfo.numActiveIOOps > 0) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ for (IndexInfo iInfo : dsInfo.indexes.values()) {
+ // TODO: This is not efficient since we flush the indexes sequentially.
+ // Think of a way to allow submitting the flush requests concurrently. We don't do them concurrently because this
+ // may lead to a deadlock scenario between the DatasetLifeCycleManager and the PrimaryIndexOperationTracker.
+ flushAndWaitForIO(dsInfo, iInfo);
+ }
+
for (IndexInfo iInfo : dsInfo.indexes.values()) {
if (iInfo.isOpen) {
- iInfo.index.deactivate(true);
+ iInfo.index.deactivate(false);
iInfo.isOpen = false;
}
assert iInfo.referenceCount == 0;
}
-
- IVirtualBufferCache vbc = getVirtualBufferCache(dsInfo.datasetID);
- used -= vbc.getNumPages() * vbc.getPageSize();
dsInfo.isOpen = false;
+
+ List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(dsInfo.datasetID);
+ for (IVirtualBufferCache vbc : vbcs) {
+ used -= vbc.getNumPages() * vbc.getPageSize();
+ }
return true;
+
}
}
return false;
}
+ private void flushAndWaitForIO(DatasetInfo dsInfo, IndexInfo iInfo) throws HyracksDataException {
+ if (iInfo.isOpen) {
+ ILSMIndexAccessor accessor = (ILSMIndexAccessor) iInfo.index.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ accessor.scheduleFlush(iInfo.index.getIOOperationCallback());
+ }
+ // Wait for the above flush op.
+ while (dsInfo.numActiveIOOps > 0) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
@Override
public synchronized void close(long resourceID) throws HyracksDataException {
int did = getDIDfromRID(resourceID);
@@ -213,15 +292,22 @@
return openIndexes;
}
- public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
+ public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
synchronized (datasetVirtualBufferCaches) {
- MultitenantVirtualBufferCache vbc = datasetVirtualBufferCaches.get(datasetID);
- if (vbc == null) {
- vbc = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(),
- storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages()));
- datasetVirtualBufferCaches.put(datasetID, vbc);
+ List<IVirtualBufferCache> vbcs = datasetVirtualBufferCaches.get(datasetID);
+ if (vbcs == null) {
+ vbcs = new ArrayList<IVirtualBufferCache>();
+ for (int i = 0; i < storageProperties.getMemoryComponentsNum(); i++) {
+ MultitenantVirtualBufferCache vbc = new MultitenantVirtualBufferCache(
+ new VirtualBufferCache(new HeapBufferAllocator(),
+ storageProperties.getMemoryComponentPageSize(),
+ storageProperties.getMemoryComponentNumPages()
+ / storageProperties.getMemoryComponentsNum()));
+ vbcs.add(vbc);
+ }
+ datasetVirtualBufferCaches.put(datasetID, vbcs);
}
- return vbc;
+ return vbcs;
}
}
@@ -229,8 +315,7 @@
synchronized (datasetOpTrackers) {
ILSMOperationTracker opTracker = datasetOpTrackers.get(datasetID);
if (opTracker == null) {
- opTracker = new PrimaryIndexOperationTracker(this, datasetID,
- LSMBTreeIOOperationCallbackFactory.INSTANCE);
+ opTracker = new PrimaryIndexOperationTracker(this, datasetID);
datasetOpTrackers.put(datasetID, opTracker);
}
@@ -245,7 +330,9 @@
}
Set<ILSMIndex> datasetIndexes = new HashSet<ILSMIndex>();
for (IndexInfo iInfo : dsInfo.indexes.values()) {
- datasetIndexes.add(iInfo.index);
+ if (iInfo.isOpen) {
+ datasetIndexes.add(iInfo.index);
+ }
}
return datasetIndexes;
}
@@ -280,6 +367,7 @@
private final Map<Long, IndexInfo> indexes;
private final int datasetID;
private long lastAccess;
+ private int numActiveIOOps;
public DatasetInfo(int datasetID) {
this.indexes = new HashMap<Long, IndexInfo>();
@@ -297,6 +385,14 @@
lastAccess = System.currentTimeMillis();
}
+ public void incrementActiveIOOps() {
+ numActiveIOOps++;
+ }
+
+ public void decrementActiveIOOps() {
+ numActiveIOOps--;
+ }
+
@Override
public int compareTo(DatasetInfo i) {
// sort by (isOpen, referenceCount, lastAccess) ascending, where true < false
@@ -376,4 +472,4 @@
}
outputStream.write(sb.toString().getBytes());
}
-}
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
index 53b9192..ef58937 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -23,38 +23,28 @@
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexInternal;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
public class PrimaryIndexOperationTracker extends BaseOperationTracker {
- private final DatasetLifecycleManager datasetLifecycleManager;
- private final IVirtualBufferCache datasetBufferCache;
- private final int datasetID;
- // Number of active operations on a ILSMIndex instance.
- private AtomicInteger numActiveOperations;
+ // Number of active operations on an ILSMIndex instance.
+ private final AtomicInteger numActiveOperations;
- public PrimaryIndexOperationTracker(DatasetLifecycleManager datasetLifecycleManager, int datasetID,
- ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
- super(ioOpCallbackFactory);
- this.datasetLifecycleManager = datasetLifecycleManager;
- this.numActiveOperations = new AtomicInteger(0);
- this.datasetID = datasetID;
- datasetBufferCache = datasetLifecycleManager.getVirtualBufferCache(datasetID);
+ public PrimaryIndexOperationTracker(DatasetLifecycleManager datasetLifecycleManager, int datasetID) {
+ super(datasetLifecycleManager, datasetID);
+ this.numActiveOperations = new AtomicInteger();
}
@Override
public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
- numActiveOperations.incrementAndGet();
-
- // Increment transactor-local active operations count.
- AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
- if (opCallback != null) {
- opCallback.incrementLocalNumActiveOperations();
+ if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
+ incrementNumActiveOperations(modificationCallback);
+ } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ datasetLifecycleManager.declareActiveIOOperation(datasetID);
}
}
@@ -62,8 +52,7 @@
public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
// Searches are immediately considered complete, because they should not prevent the execution of flushes.
- if ((searchCallback != null && searchCallback != NoOpOperationCallback.INSTANCE)
- || opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ if (opType == LSMOperationType.SEARCH || opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
completeOperation(index, opType, searchCallback, modificationCallback);
}
}
@@ -71,50 +60,65 @@
@Override
public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
- int nActiveOps = numActiveOperations.decrementAndGet();
- // Decrement transactor-local active operations count.
- AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
- if (opCallback != null) {
- opCallback.decrementLocalNumActiveOperations();
- }
- if (opType != LSMOperationType.FLUSH) {
- flushIfFull(nActiveOps);
+ if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
+ decrementNumActiveOperations(modificationCallback);
+ flushIfFull();
+ } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ datasetLifecycleManager.undeclareActiveIOOperation(datasetID);
}
}
- private void flushIfFull(int nActiveOps) throws HyracksDataException {
- // If we need a flush, and this is the last completing operation, then schedule the flush.
- if (datasetBufferCache.isFull() && nActiveOps == 0) {
- Set<ILSMIndex> indexes = datasetLifecycleManager.getDatasetIndexes(datasetID);
- for (ILSMIndex lsmIndex : indexes) {
- ILSMIndexAccessor accessor = (ILSMIndexAccessor) lsmIndex.createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- accessor.scheduleFlush(((BaseOperationTracker) lsmIndex.getOperationTracker()).getIOOperationCallback());
+ private void flushIfFull() throws HyracksDataException {
+ // If we need a flush, and this is the last completing operation, then schedule the flush.
+ // TODO: Is there a better way to check if we need to flush instead of communicating with the datasetLifecycleManager each time?
+ Set<ILSMIndex> indexes = datasetLifecycleManager.getDatasetIndexes(datasetID);
+ boolean needsFlush = false;
+ for (ILSMIndex lsmIndex : indexes) {
+ if (((ILSMIndexInternal) lsmIndex).hasFlushRequestForCurrentMutableComponent()) {
+ needsFlush = true;
+ break;
}
-
+ }
+ synchronized (this) {
+ if (needsFlush && numActiveOperations.get() == 0) {
+ for (ILSMIndex lsmIndex : indexes) {
+ ILSMIndexAccessor accessor = (ILSMIndexAccessor) lsmIndex.createAccessor(
+ NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ accessor.scheduleFlush(lsmIndex.getIOOperationCallback());
+ }
+ }
}
}
public void exclusiveJobCommitted() throws HyracksDataException {
numActiveOperations.set(0);
- flushIfFull(0);
- }
-
- private AbstractOperationCallback getOperationCallback(ISearchOperationCallback searchCallback,
- IModificationOperationCallback modificationCallback) {
-
- if (modificationCallback == NoOpOperationCallback.INSTANCE || modificationCallback == null) {
- return null;
- }
- if (searchCallback != null && searchCallback != NoOpOperationCallback.INSTANCE) {
- return (AbstractOperationCallback) searchCallback;
- } else {
- return (AbstractOperationCallback) modificationCallback;
- }
+ flushIfFull();
}
public int getNumActiveOperations() {
return numActiveOperations.get();
}
-}
+ private void incrementNumActiveOperations(IModificationOperationCallback modificationCallback) {
+ //modificationCallback can be NoOpOperationCallback when redo/undo operations are executed.
+ if (modificationCallback != NoOpOperationCallback.INSTANCE) {
+ numActiveOperations.incrementAndGet();
+ ((AbstractOperationCallback) modificationCallback).incrementLocalNumActiveOperations();
+ }
+ }
+
+ private void decrementNumActiveOperations(IModificationOperationCallback modificationCallback) {
+ //modificationCallback can be NoOpOperationCallback when redo/undo operations are executed.
+ if (modificationCallback != NoOpOperationCallback.INSTANCE) {
+ numActiveOperations.decrementAndGet();
+ ((AbstractOperationCallback) modificationCallback).decrementLocalNumActiveOperations();
+ }
+ }
+
+ public void cleanupNumActiveOperationsForAbortedJob(AbstractOperationCallback callback) {
+ int delta = callback.getLocalNumActiveOperations() * -1;
+ numActiveOperations.getAndAdd(delta);
+ callback.resetLocalNumActiveOperations();
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index e850619..26a79c3 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -47,7 +47,6 @@
if (tupleFilter != null) {
frameTuple.reset(accessor, i);
if (!tupleFilter.accept(frameTuple)) {
- lsmAccessor.noOp();
continue;
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
index fcc6dad..684068b 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
@@ -17,35 +17,52 @@
import java.util.List;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
public abstract class AbstractLSMIOOperationCallback implements ILSMIOOperationCallback {
- protected final BaseOperationTracker opTracker;
+ protected long firstLSN;
+ protected long lastLSN;
+ protected long[] immutableLastLSNs;
+ protected int readIndex;
+ protected int writeIndex;
- public AbstractLSMIOOperationCallback(BaseOperationTracker opTracker) {
- this.opTracker = opTracker;
+ public AbstractLSMIOOperationCallback() {
+ resetLSNs();
}
@Override
- public void beforeOperation() {
+ public void setNumOfMutableComponents(int count) {
+ immutableLastLSNs = new long[count];
+ readIndex = 0;
+ writeIndex = 0;
+ }
+
+ @Override
+ public void beforeOperation(LSMOperationType opType) {
+ if (opType == LSMOperationType.FLUSH) {
+ synchronized (this) {
+ immutableLastLSNs[writeIndex] = lastLSN;
+ writeIndex = (writeIndex + 1) % immutableLastLSNs.length;
+ resetLSNs();
+ }
+ }
+ }
+
+ @Override
+ public void afterFinalize(LSMOperationType opType, ILSMComponent newComponent) {
// Do nothing.
}
- @Override
- public void afterFinalize(ILSMComponent newComponent) {
- opTracker.resetLSNs();
- }
-
- protected abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
+ public abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
protected void putLSNIntoMetadata(ITreeIndex treeIndex, List<ILSMComponent> oldComponents)
throws HyracksDataException {
@@ -80,4 +97,25 @@
bufferCache.unpin(metadataPage);
}
}
+
+ protected void resetLSNs() {
+ firstLSN = -1;
+ lastLSN = -1;
+ }
+
+ public void updateLastLSN(long lastLSN) {
+ if (firstLSN == -1) {
+ firstLSN = lastLSN;
+ }
+ this.lastLSN = Math.max(this.lastLSN, lastLSN);
+ }
+
+ public long getFirstLSN() {
+ return firstLSN;
+ }
+
+ public long getLastLSN() {
+ return lastLSN;
+ }
+
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
index 40377f4..8e9b44e 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
@@ -17,37 +17,41 @@
import java.util.List;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeImmutableComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponent;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
- public LSMBTreeIOOperationCallback(BaseOperationTracker opTracker) {
- super(opTracker);
+ public LSMBTreeIOOperationCallback() {
+ super();
}
@Override
- public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+ public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
throws HyracksDataException {
- if (oldComponents != null && newComponent != null) {
- LSMBTreeImmutableComponent btreeComponent = (LSMBTreeImmutableComponent) newComponent;
+ if (newComponent != null) {
+ LSMBTreeDiskComponent btreeComponent = (LSMBTreeDiskComponent) newComponent;
putLSNIntoMetadata(btreeComponent.getBTree(), oldComponents);
}
}
@Override
- protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
- if (oldComponents == null) {
+ public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+ if (diskComponents == null) {
// Implies a flush IO operation.
- return opTracker.getLastLSN();
+ synchronized (this) {
+ long lsn = immutableLastLSNs[readIndex];
+ readIndex = (readIndex + 1) % immutableLastLSNs.length;
+ return lsn;
+ }
}
- // Get max LSN from the oldComponents. Implies a merge IO operation.
+ // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
long maxLSN = -1;
- for (ILSMComponent c : oldComponents) {
- BTree btree = ((LSMBTreeImmutableComponent) c).getBTree();
+ for (ILSMComponent c : diskComponents) {
+ BTree btree = ((LSMBTreeDiskComponent) c).getBTree();
maxLSN = Math.max(getTreeIndexLSN(btree), maxLSN);
}
return maxLSN;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
index 92ba9ec..1028015 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
@@ -15,7 +15,6 @@
package edu.uci.ics.asterix.common.ioopcallbacks;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -29,7 +28,7 @@
}
@Override
- public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
- return new LSMBTreeIOOperationCallback((BaseOperationTracker) syncObj);
+ public ILSMIOOperationCallback createIOOperationCallback() {
+ return new LSMBTreeIOOperationCallback();
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
index eb9878c..5532f97 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
@@ -17,36 +17,40 @@
import java.util.List;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexImmutableComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexDiskComponent;
public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
- public LSMInvertedIndexIOOperationCallback(BaseOperationTracker opTracker) {
- super(opTracker);
+ public LSMInvertedIndexIOOperationCallback() {
+ super();
}
@Override
- public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+ public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
throws HyracksDataException {
- if (oldComponents != null && newComponent != null) {
- LSMInvertedIndexImmutableComponent invIndexComponent = (LSMInvertedIndexImmutableComponent) newComponent;
+ if (newComponent != null) {
+ LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) newComponent;
putLSNIntoMetadata(invIndexComponent.getDeletedKeysBTree(), oldComponents);
}
}
@Override
- protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
- if (oldComponents == null) {
+ public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+ if (diskComponents == null) {
// Implies a flush IO operation.
- return opTracker.getLastLSN();
+ synchronized (this) {
+ long lsn = immutableLastLSNs[readIndex];
+ readIndex = (readIndex + 1) % immutableLastLSNs.length;
+ return lsn;
+ }
}
- // Get max LSN from the oldComponents. Implies a merge IO operation.
+ // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
long maxLSN = -1;
- for (Object o : oldComponents) {
- LSMInvertedIndexImmutableComponent invIndexComponent = (LSMInvertedIndexImmutableComponent) o;
+ for (Object o : diskComponents) {
+ LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) o;
maxLSN = Math.max(getTreeIndexLSN(invIndexComponent.getDeletedKeysBTree()), maxLSN);
}
return maxLSN;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
index c20cdb3..5dc0c0b 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
@@ -15,7 +15,6 @@
package edu.uci.ics.asterix.common.ioopcallbacks;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -29,7 +28,7 @@
}
@Override
- public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
- return new LSMInvertedIndexIOOperationCallback((BaseOperationTracker) syncObj);
+ public ILSMIOOperationCallback createIOOperationCallback() {
+ return new LSMInvertedIndexIOOperationCallback();
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
index 63016f1..1497e17 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
@@ -17,37 +17,41 @@
import java.util.List;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeImmutableComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeDiskComponent;
public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
- public LSMRTreeIOOperationCallback(BaseOperationTracker opTracker) {
- super(opTracker);
+ public LSMRTreeIOOperationCallback() {
+ super();
}
@Override
- public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+ public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
throws HyracksDataException {
- if (oldComponents != null && newComponent != null) {
- LSMRTreeImmutableComponent rtreeComponent = (LSMRTreeImmutableComponent) newComponent;
+ if (newComponent != null) {
+ LSMRTreeDiskComponent rtreeComponent = (LSMRTreeDiskComponent) newComponent;
putLSNIntoMetadata(rtreeComponent.getRTree(), oldComponents);
putLSNIntoMetadata(rtreeComponent.getBTree(), oldComponents);
}
}
@Override
- protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
- if (oldComponents == null) {
+ public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+ if (diskComponents == null) {
// Implies a flush IO operation.
- return opTracker.getLastLSN();
+ synchronized (this) {
+ long lsn = immutableLastLSNs[readIndex];
+ readIndex = (readIndex + 1) % immutableLastLSNs.length;
+ return lsn;
+ }
}
- // Get max LSN from the oldComponents. Implies a merge IO operation.
+ // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
long maxLSN = -1;
- for (Object o : oldComponents) {
- LSMRTreeImmutableComponent rtreeComponent = (LSMRTreeImmutableComponent) o;
+ for (Object o : diskComponents) {
+ LSMRTreeDiskComponent rtreeComponent = (LSMRTreeDiskComponent) o;
maxLSN = Math.max(getTreeIndexLSN(rtreeComponent.getRTree()), maxLSN);
}
return maxLSN;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
index 0cd2539..841a1d5 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
@@ -15,7 +15,6 @@
package edu.uci.ics.asterix.common.ioopcallbacks;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -29,7 +28,7 @@
}
@Override
- public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
- return new LSMRTreeIOOperationCallback((BaseOperationTracker) syncObj);
+ public ILSMIOOperationCallback createIOOperationCallback() {
+ return new LSMRTreeIOOperationCallback();
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
index c951826..c549e7d 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
@@ -28,8 +28,8 @@
protected final int[] primaryKeyFields;
protected final ITransactionContext txnCtx;
protected final ILockManager lockManager;
- protected final AtomicInteger transactorLocalNumActiveOperations;
protected final long[] longHashes;
+ protected final AtomicInteger transactorLocalNumActiveOperations;
public AbstractOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
ILockManager lockManager) {
@@ -45,6 +45,10 @@
MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
return Math.abs((int) longHashes[0]);
}
+
+ public void resetLocalNumActiveOperations() {
+ transactorLocalNumActiveOperations.set(0);
+ }
public int getLocalNumActiveOperations() {
return transactorLocalNumActiveOperations.get();
@@ -57,4 +61,5 @@
public void decrementLocalNumActiveOperations() {
transactorLocalNumActiveOperations.decrementAndGet();
}
+
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java
deleted file mode 100644
index a10bf08..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support. This class is not "thread-safe". For performance
- * concerns, it is required for multiple writers to be able to write to the
- * buffer concurrently and that a writer is never blocked by another writer. The
- * users of this class must ensure that two concurrent writers get to write in
- * exclusive areas in the buffer. A reader and writer may or may not conflict
- * with each other. For example, reading of logs during roll back of a
- * transaction t1 does not conflict with writing of logs by another transaction
- * t2 as they are concerned with exclusive areas of the buffer. On the contrary,
- * a flushing the buffer to disk conflicts with a reader reading the buffer.
- * Appropriate locks are taken on the Buffer in the application logic and not
- * directly imposed by synchronized methods.
- */
-
-public class Buffer implements IBuffer {
-
- ByteBuffer buffer;
-
- public Buffer(ByteBuffer buffer) {
- this.buffer = buffer;
- }
-
- protected Buffer() {
- }
-
- public void setBuffer(ByteBuffer buffer) {
- this.buffer = buffer;
- }
-
- @Override
- public byte getByte(int offset) {
- return buffer.get(offset);
- }
-
- @Override
- public byte getByte() {
- return buffer.get();
- }
-
- @Override
- public void getBytes(byte[] bytes, int offset, int size) {
- System.arraycopy(buffer.array(), offset, bytes, 0, size);
- }
-
- @Override
- public int getSize() {
- return buffer.capacity();
- }
-
- @Override
- public int readInt() {
- return buffer.getInt();
- }
-
- @Override
- public int readInt(int offset) {
- return buffer.getInt(offset);
- }
-
- @Override
- public long readLong(int offset) {
- return buffer.getLong(offset);
- }
-
- @Override
- public void put(byte b) {
- buffer.put(b);
- }
-
- @Override
- public void put(int offset, byte b) {
- buffer.put(offset, b);
- }
-
- @Override
- public void put(byte[] bytes, int start, int length) {
- buffer.put(bytes, start, length);
-
- }
-
- @Override
- public void put(byte[] bytes) {
- buffer.put(bytes);
- }
-
- @Override
- public void writeInt(int value) {
- buffer.putInt(value);
- }
-
- @Override
- public void writeInt(int offset, int value) {
- buffer.putInt(offset, value);
-
- }
-
- @Override
- public void writeLong(long value) {
- buffer.putLong(value);
- }
-
- @Override
- public void writeLong(int offset, long value) {
- buffer.putLong(offset, value);
-
- }
-
- @Override
- public byte[] getArray() {
- return buffer.array();
- }
-
- @Override
- public void erase() {
- Arrays.fill(buffer.array(), (byte) 0);
- }
-
- @Override
- public ByteBuffer getByteBuffer() {
- return buffer;
- }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
deleted file mode 100644
index 7ef378e..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-/**
- * Represent a buffer that is backed by a physical file. Provider custom APIs
- * for accessing a chunk of the underlying file.
- */
-public class FileBasedBuffer extends Buffer implements IFileBasedBuffer {
-
- private String filePath;
- private FileChannel fileChannel;
- private RandomAccessFile raf;
- private int bufferSize;
-
- private int bufferLastFlushOffset;
- private int bufferNextWriteOffset;
- private final int diskSectorSize;
-
- private final ReadWriteLock latch;
- private final AtomicInteger referenceCount;
-
- public FileBasedBuffer(String filePath, long offset, int bufferSize, int diskSectorSize) throws IOException {
- this.filePath = filePath;
- buffer = ByteBuffer.allocate(bufferSize);
- raf = new RandomAccessFile(new File(filePath), "rw");
- fileChannel = raf.getChannel();
- fileChannel.position(offset);
- fileChannel.read(buffer);
- buffer.position(0);
- this.bufferSize = bufferSize;
- buffer.limit(bufferSize);
- bufferLastFlushOffset = 0;
- bufferNextWriteOffset = 0;
- this.diskSectorSize = diskSectorSize;
- latch = new ReentrantReadWriteLock(true);
- referenceCount = new AtomicInteger(0);
- }
-
- public String getFilePath() {
- return filePath;
- }
-
- public void setFilePath(String filePath) {
- this.filePath = filePath;
- }
-
- @Override
- public int getSize() {
- return bufferSize;
- }
-
- public void clear() {
- buffer.clear();
- }
-
- @Override
- public void flush() throws IOException {
- //flush
- int pos = bufferLastFlushOffset;
- int limit = (((bufferNextWriteOffset - 1) / diskSectorSize) + 1) * diskSectorSize;
- buffer.position(pos);
- buffer.limit(limit);
- fileChannel.write(buffer);
- fileChannel.force(true);
-
- //update variables
- bufferLastFlushOffset = limit;
- bufferNextWriteOffset = limit;
- buffer.limit(bufferSize);
- }
-
- @Override
- public void put(byte b) {
- buffer.put(b);
- }
-
- @Override
- public void put(int offset, byte b) {
- buffer.put(offset, b);
- }
-
- @Override
- public void put(byte[] bytes, int start, int length) {
- buffer.put(bytes, start, length);
- }
-
- @Override
- public void put(byte[] bytes) {
- buffer.put(bytes);
- }
-
- @Override
- public void writeInt(int value) {
- buffer.putInt(value);
- }
-
- @Override
- public void writeInt(int index, int value) {
- buffer.putInt(index, value);
- }
-
- @Override
- public void writeLong(long value) {
- buffer.putLong(value);
- }
-
- @Override
- public void writeLong(int index, long value) {
- buffer.putLong(index, value);
- }
-
- /**
- * Resets the buffer with content (size as specified) from a given file
- * starting at offset.
- */
- @Override
- public void reset(String filePath, long diskNextWriteOffset, int bufferSize) throws IOException {
- if (!filePath.equals(this.filePath)) {
- raf.close();//required?
- fileChannel.close();
- raf = new RandomAccessFile(filePath, "rw");
- this.filePath = filePath;
- }
- fileChannel = raf.getChannel();
- fileChannel.position(diskNextWriteOffset);
- erase();
- buffer.position(0);
- buffer.limit(bufferSize);
- this.bufferSize = bufferSize;
-
- bufferLastFlushOffset = 0;
- bufferNextWriteOffset = 0;
- }
-
- @Override
- public void close() throws IOException {
- fileChannel.close();
- }
-
- @Override
- public void open(String filePath, long offset, int bufferSize) throws IOException {
- raf = new RandomAccessFile(filePath, "rw");
- fileChannel = raf.getChannel();
- fileChannel.position(offset);
- erase();
- buffer.position(0);
- buffer.limit(bufferSize);
- this.bufferSize = bufferSize;
- bufferLastFlushOffset = 0;
- bufferNextWriteOffset = 0;
- }
-
- @Override
- public long getDiskNextWriteOffset() throws IOException {
- return fileChannel.position();
- }
-
- @Override
- public void setDiskNextWriteOffset(long offset) throws IOException {
- fileChannel.position(offset);
- }
-
- @Override
- public int getBufferLastFlushOffset() {
- return bufferLastFlushOffset;
- }
-
- @Override
- public void setBufferLastFlushOffset(int offset) {
- this.bufferLastFlushOffset = offset;
- }
-
- @Override
- public int getBufferNextWriteOffset() {
- synchronized (fileChannel) {
- return bufferNextWriteOffset;
- }
- }
-
- @Override
- public void setBufferNextWriteOffset(int offset) {
- synchronized (fileChannel) {
- if (bufferNextWriteOffset < offset) {
- bufferNextWriteOffset = offset;
- }
- }
- }
-
- @Override
- public void acquireWriteLatch() {
- latch.writeLock().lock();
- }
-
- @Override
- public void releaseWriteLatch() {
- latch.writeLock().unlock();
- }
-
- @Override
- public void acquireReadLatch() {
- latch.readLock().lock();
- }
-
- @Override
- public void releaseReadLatch() {
- latch.readLock().unlock();
- }
-
- @Override
- public void incRefCnt() {
- referenceCount.incrementAndGet();
- }
-
- @Override
- public void decRefCnt() {
- referenceCount.decrementAndGet();
- }
-
- @Override
- public int getRefCnt() {
- return referenceCount.get();
- }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
deleted file mode 100644
index 64beb86..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.io.File;
-import java.io.IOException;
-
-/**
- * A utility class that provides operations on files such as creation and
- * loading content in a buffer. It also provides API for creating directories
- */
-public class FileUtil {
-
- public static final String lineSeparator = System.getProperty("line.separator");
-
- public static boolean createFileIfNotExists(String path) throws IOException {
- File file = new File(path);
- File parentFile = file.getParentFile();
- if (parentFile != null) {
- parentFile.mkdirs();
- }
- return file.createNewFile();
- }
-
- public static boolean createNewDirectory(String path) throws IOException {
- return (new File(path)).mkdir();
- }
-
- public static IFileBasedBuffer getFileBasedBuffer(String filePath, long offset, int bufferSize, int diskSectorSize) throws IOException {
- IFileBasedBuffer fileBasedBuffer = new FileBasedBuffer(filePath, offset, bufferSize, diskSectorSize);
- return fileBasedBuffer;
- }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
index 05ac025..2638c9f 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
@@ -14,9 +14,10 @@
*/
package edu.uci.ics.asterix.common.transactions;
+import java.util.List;
+
import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -42,14 +43,6 @@
public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID);
- public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider(boolean isPrimary);
-
- public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider();
-
- public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider();
-
- public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider();
-
public ILSMIOOperationScheduler getLSMIOScheduler();
public ILocalResourceRepository getLocalResourceRepository();
@@ -58,5 +51,5 @@
public IIOManager getIOManager();
- public IVirtualBufferCache getVirtualBufferCache(int datasetID);
+ public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID);
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
deleted file mode 100644
index fd4af81..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.nio.ByteBuffer;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support.
- */
-public interface IBuffer {
-
- public byte getByte(int offset);
-
- public int getSize();
-
- public byte getByte();
-
- public void getBytes(byte[] bytes, int start, int size);
-
- public int readInt();
-
- public int readInt(int offset);
-
- public long readLong(int offset);
-
- public void put(byte b);
-
- public void put(int offset, byte b);
-
- public void put(byte[] bytes, int start, int length);
-
- public void put(byte[] bytes);
-
- public void writeInt(int value);
-
- public void writeLong(long value);
-
- public void writeInt(int offset, int value);
-
- public void writeLong(int offset, long value);
-
- public byte[] getArray();
-
- public void erase();
-
- public ByteBuffer getByteBuffer();
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
deleted file mode 100644
index 1c8cb76..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * Represents a closeable resource that implements a close(@see
- * TransactionContext) method. When a transaction commits/aborts, the close
- * method is called on each of the ICloseable resources that were involved in
- * the transaction. This gives an opportunity to release all resources and do a
- * cleanup. An example of ICloseable is the @see TreeLogger.
- */
-public interface ICloseable {
-
- /**
- * This method is invoked at the commit/abort of a transaction that involved
- * a ICloseable resource. It is used to do a clean up by the involved
- * resource before the transaction ends.
- *
- * @param context
- * @throws ACIDException
- */
- public void close(ITransactionContext context) throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
deleted file mode 100644
index 1d466e3..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.io.IOException;
-
-/**
- * Represent a buffer that is backed by a physical file. Provides custom APIs
- * for accessing a chunk of the underlying file.
- */
-
-public interface IFileBasedBuffer extends IBuffer {
-
- public void flush() throws IOException;
-
- /**
- * Resets the buffer with content (size as specified) from a given file
- * starting at offset.
- */
- public void reset(String filePath, long offset, int size) throws IOException;
-
- public long getDiskNextWriteOffset() throws IOException;
-
- public void setDiskNextWriteOffset(long writePosition) throws IOException;
-
- public void close() throws IOException;
-
- public void open(String filePath, long offset, int size) throws IOException;
-
- public int getBufferLastFlushOffset();
-
- public void setBufferLastFlushOffset(int offset);
-
- public int getBufferNextWriteOffset();
-
- public void setBufferNextWriteOffset(int offset);
-
- public void acquireWriteLatch();
-
- public void releaseWriteLatch();
-
- public void acquireReadLatch();
-
- public void releaseReadLatch();
-
- public void incRefCnt();
-
- public void decRefCnt();
-
- public int getRefCnt();
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
index 698a788..a752afa 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
@@ -64,20 +64,9 @@
* @param txnContext
* @throws ACIDException
* TODO
- * @return true if the lock count is 0, false otherwise.
+ * @return
*/
- public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
- throws ACIDException;
-
- /**
- * @param datasetId
- * @param entityHashValue
- * @param txnContext
- * @throws ACIDException
- * TODO
- * @return true if the lock count is 0, false otherwise.
- */
- public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
+ public void unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
throws ACIDException;
/**
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
deleted file mode 100644
index 5c5c149..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * Represents a predicate or a filter that needs to be applied when selectively
- * retrieving logs.
- */
-public interface ILogFilter {
-
- public boolean accept(IBuffer buffer, long startOffset, int length);
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
index 9cc4d0d..27a91a4 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
@@ -14,73 +14,12 @@
*/
package edu.uci.ics.asterix.common.transactions;
-import java.io.IOException;
-
import edu.uci.ics.asterix.common.exceptions.ACIDException;
public interface ILogManager {
- /**
- * @param logType
- * @param context
- * @param datasetId
- * @param PKHashValue
- * @param resourceId
- * @param resourceMgrId
- * @param logContentSize
- * @param reusableLogContentObject
- * @param logger
- * @param logicalLogLocator
- * @throws ACIDException
- */
- void log(byte logType, ITransactionContext context, int datasetId, int PKHashValue, long resourceId,
- byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
- LogicalLogLocator logicalLogLocator) throws ACIDException;
+ public void log(ILogRecord logRecord) throws ACIDException;
- /**
- * @param physicalLogLocator
- * specifies the physical location from where the logs need to be
- * read
- * @param logFilter
- * specifies the filtering criteria for the retrieved logs
- * @return LogCursor an iterator for the retrieved logs
- * @throws ACIDException
- */
- public ILogCursor readLog(PhysicalLogLocator physicalLogLocator, ILogFilter logFilter) throws IOException,
- ACIDException;
-
- /**
- * @param lsnValue
- * TODO
- * @param logicalLogLocator
- * TODO
- * @throws ACIDException
- */
- public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException;
-
- /**
- * Retrieves the configuration parameters of the ILogManager
- *
- * @return LogManagerProperties: the configuration parameters for the
- * ILogManager
- */
- public LogManagerProperties getLogManagerProperties();
-
- /**
- * Returns the ILogRecordHelper instance associated with this ILogManager
- * instance
- *
- * @return ILogRecordHelper: the utility (class) for writing/reading log
- * header.
- */
- public ILogRecordHelper getLogRecordHelper();
-
- /**
- * Returns the Transaction Provider associated with this ILogManager
- * instance
- *
- * @return TransactionSubsystem
- */
- public ITransactionSubsystem getTransactionSubsystem();
+ public ILogReader getLogReader(boolean isRecoveryMode);
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
similarity index 78%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
index 08940a7..be205a9 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
@@ -14,9 +14,10 @@
*/
package edu.uci.ics.asterix.common.transactions;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
+public interface ILogPage {
-public interface ILoggerRepository {
+ public void append(ILogRecord logRecord, long appendLsn);
- public ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException;
-}
+ public void flush();
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
similarity index 70%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
index e455d3cc..fa92865 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
@@ -14,18 +14,18 @@
*/
package edu.uci.ics.asterix.common.transactions;
-import java.io.IOException;
-
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-/**
- * Provides a cursor over the logs created to date.
- */
-public interface ILogCursor {
+public interface ILogReader {
- public boolean next(LogicalLogLocator currentLogLocator) throws IOException, ACIDException;
+ public void initializeScan(long beginLSN) throws ACIDException;
- public ILogFilter getLogFilter();
+ //for scanning
+ public ILogRecord next() throws ACIDException;
-}
+ //for random reading
+ public ILogRecord read(long readLSN) throws ACIDException;
+
+ public void close() throws ACIDException;
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
new file mode 100644
index 0000000..3068867
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.common.transactions;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface ILogRecord {
+
+ public static final int JOB_TERMINATE_LOG_SIZE = 13; //JOB_COMMIT or ABORT log type
+ public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 25;
+ public static final int UPDATE_LOG_BASE_SIZE = 60;
+
+ public boolean readLogRecord(ByteBuffer buffer);
+
+ public void writeLogRecord(ByteBuffer buffer);
+
+ public void formJobTerminateLogRecord(ITransactionContext txnCtx, boolean isCommit);
+
+ public void formEntityCommitLogRecord(ITransactionContext txnCtx, int datasetId, int PKHashValue,
+ ITupleReference tupleReference, int[] primaryKeyFields);
+
+ public ITransactionContext getTxnCtx();
+
+ public void setTxnCtx(ITransactionContext txnCtx);
+
+ public boolean isFlushed();
+
+ public void isFlushed(boolean isFlushed);
+
+ public byte getLogType();
+
+ public void setLogType(byte logType);
+
+ public int getJobId();
+
+ public void setJobId(int jobId);
+
+ public int getDatasetId();
+
+ public void setDatasetId(int datasetId);
+
+ public int getPKHashValue();
+
+ public void setPKHashValue(int PKHashValue);
+
+ public long getPrevLSN();
+
+ public void setPrevLSN(long prevLsn);
+
+ public long getResourceId();
+
+ public void setResourceId(long resourceId);
+
+ public byte getResourceType();
+
+ public void setResourceType(byte resourceType);
+
+ public int getLogSize();
+
+ public void setLogSize(int logSize);
+
+ public byte getNewOp();
+
+ public void setNewOp(byte newOp);
+
+ public int getNewValueSize();
+
+ public void setNewValueSize(int newValueSize);
+
+ public ITupleReference getNewValue();
+
+ public void setNewValue(ITupleReference newValue);
+
+ public byte getOldOp();
+
+ public void setOldOp(byte oldOp);
+
+ public int getOldValueSize();
+
+ public void setOldValueSize(int oldValueSize);
+
+ public ITupleReference getOldValue();
+
+ public void setOldValue(ITupleReference oldValue);
+
+ public long getChecksum();
+
+ public void setChecksum(long checksum);
+
+ public long getLSN();
+
+ public void setLSN(long LSN);
+
+ public String getLogRecordForDisplay();
+
+ public void computeAndSetLogSize();
+
+ public int getPKValueSize();
+
+ public ITupleReference getPKValue();
+
+ public void setPKFields(int[] primaryKeyFields);
+
+ public void computeAndSetPKValueSize();
+
+ public void setPKValue(ITupleReference PKValue);
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
deleted file mode 100644
index a299a0d..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.common.transactions;
-
-/**
- * Helper class for writing/reading of log header and checksum as well as
- * validating log record by checksum comparison. Every ILogManager
- * implementation has an associated ILogRecordHelper implementation.
- */
-
-public interface ILogRecordHelper {
-
- public byte getLogType(LogicalLogLocator logicalLogLocator);
-
- public int getJobId(LogicalLogLocator logicalLogLocator);
-
- public int getDatasetId(LogicalLogLocator logicalLogLocator);
-
- public int getPKHashValue(LogicalLogLocator logicalLogLocator);
-
- public PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator);
-
- public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator);
-
- public long getResourceId(LogicalLogLocator logicalLogLocator);
-
- public byte getResourceMgrId(LogicalLogLocator logicalLogLocater);
-
- public int getLogContentSize(LogicalLogLocator logicalLogLocater);
-
- public long getLogChecksum(LogicalLogLocator logicalLogLocator);
-
- public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator);
-
- public int getLogContentEndPos(LogicalLogLocator logicalLogLocator);
-
- public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
-
- public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
- int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
- int logRecordSize);
-
- public boolean validateLogRecord(LogicalLogLocator logicalLogLocator);
-
- public int getLogRecordSize(byte logType, int logBodySize);
-
- public int getLogHeaderSize(byte logType);
-
- public int getLogChecksumSize();
-
- public int getCommitLogSize();
-
-}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java
deleted file mode 100644
index 939f8e5..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * An interface providing call back APIs that are invoked {@link ILogManager} for providing the content for the log record and doing any pre/post
- * processing.
- */
-public interface ILogger {
-
- public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
- throws ACIDException;
-
- public void log(ITransactionContext context, final LogicalLogLocator logicalLogLocator, int logContentSize,
- ReusableLogContentObject reusableLogContentObject) throws ACIDException;
-
- public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
- throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
index be41a6b..6d2f3cb 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
@@ -17,7 +17,7 @@
import java.io.IOException;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
/**
* Provides API for failure recovery. Failure could be at application level and
@@ -33,6 +33,12 @@
HEALTHY,
CORRUPTED
}
+
+ public class ResourceType {
+ public static final byte LSM_BTREE = 0;
+ public static final byte LSM_RTREE = 1;
+ public static final byte LSM_INVERTED_INDEX = 2;
+ }
/**
* Returns the state of the system.
@@ -69,5 +75,5 @@
*/
public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException;
- public void checkpoint(boolean isSharpCheckpoint) throws ACIDException;
+ public void checkpoint(boolean isSharpCheckpoint) throws ACIDException, HyracksDataException;
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
deleted file mode 100644
index 36ce317..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-
-/**
- * Provides APIs for undo or redo of an operation on a resource.
- */
-public interface IResourceManager {
-
- public class ResourceType {
- public static final byte LSM_BTREE = 1;
- public static final byte LSM_RTREE = 2;
- public static final byte LSM_INVERTED_INDEX = 3;
- }
-
- /**
- * Returns the unique identifier for the resource manager.
- *
- * @return a unique identifier for the resource manager. The number of
- * resource managers in the system are expected to be handful and
- * can be uniquely identified by using a single byte as an id.
- */
- public byte getResourceManagerId();
-
- /**
- * Undo the operation corresponding to a given log record.
- *
- * @param logRecordHelper
- * (@see ILogRecordHelper) An implementation of the
- * ILogRecordHelper interface that is used to parse the log
- * record and extract useful information from the content.
- * @param LogicalLogLocator
- * (@see LogicalLogLocator) The locationof the log record that
- * needs to be undone.
- * @throws ACIDException
- */
- public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logicalLogLocator) throws ACIDException;
-
- /**
- * Redo the operation corresponding to a given log record.
- *
- * @param logRecordHelper
- * (@see ILogRecordHelper) An implementation of the
- * ILogRecordHelper interface that is used to parse the log
- * record and extract useful information from the content.
- * @param LogicalLogLocator
- * (@see LogicalLogLocator) The locationof the log record that
- * needs to be undone.
- * @throws ACIDException
- */
- public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
index 3ebb963..dc33e69 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
@@ -14,61 +14,38 @@
*/
package edu.uci.ics.asterix.common.transactions;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
public interface ITransactionContext {
- public void registerIndexAndCallback(ILSMIndex index, AbstractOperationCallback callback);
-
- public void updateLastLSNForIndexes(long lastLSN);
-
- public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException;
-
- public int getActiveOperationCountOnIndexes() throws HyracksDataException;
-
- public LogicalLogLocator getFirstLogLocator();
-
- public LogicalLogLocator getLastLogLocator();
-
- public void addCloseableResource(ICloseable resource);
+ public void registerIndexAndCallback(long resourceId, ILSMIndex index, AbstractOperationCallback callback,
+ boolean isPrimaryIndex);
public JobId getJobId();
- public void setStartWaitTime(long time);
+ public void setTimeout(boolean isTimeout);
- public long getStartWaitTime();
+ public boolean isTimeout();
- public void setStatus(int status);
+ public void setTxnState(int txnState);
- public int getStatus();
+ public int getTxnState();
- public void setTxnState(TransactionState txnState);
+ public long getFirstLSN();
- public TransactionState getTxnState();
+ public long getLastLSN();
- public void releaseResources() throws ACIDException;
+ public void setLastLSN(long resourceId, long LSN);
- public void setLastLSN(long lsn);
+ public boolean isWriteTxn();
- public TransactionType getTransactionType();
-
- public void setTransactionType(TransactionType transactionType);
+ public void setWriteTxn(boolean isWriterTxn);
public String prettyPrint();
- public static final long INVALID_TIME = -1l; // used for showing a
- // transaction is not waiting.
- public static final int ACTIVE_STATUS = 0;
- public static final int TIMED_OUT_STATUS = 1;
+ public void setMetadataTransaction(boolean isMetadataTxn);
- public enum TransactionType {
- READ,
- READ_WRITE
- }
+ public boolean isMetadataTransaction();
- public void setExclusiveJobLevelCommit();
-
+ public void notifyOptracker(boolean isJobLevelCommit);
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
index 9a99dc9..ffd4cc2 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
@@ -29,12 +29,10 @@
* transaction has committed. ABORTED: The transaction has aborted.
* TIMED_OUT: The transaction has timed out waiting to acquire a lock.
*/
- public enum TransactionState {
- ACTIVE,
- COMMITTED,
- ABORTED,
- TIMED_OUT,
- };
+ public static final int ACTIVE = 0;
+ public static final int COMMITTED = 1;
+ public static final int ABORTED = 2;
+ public static final int TIMED_OUT = 3;
/**
* Begins a transaction identified by a transaction id and returns the
@@ -54,10 +52,11 @@
*
* @param jobId
* a unique value for the transaction id.
+ * @param createIfNotExist TODO
* @return
* @throws ACIDException
*/
- public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException;
+ public ITransactionContext getTransactionContext(JobId jobId, boolean createIfNotExist) throws ACIDException;
/**
* Commits a transaction.
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
index 63ee5d0..b6c934a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.asterix.common.transactions;
-
public interface ITransactionSubsystem {
public ILogManager getLogManager();
@@ -25,10 +24,6 @@
public IRecoveryManager getRecoveryManager();
- public TransactionalResourceManagerRepository getTransactionalResourceRepository();
-
- public ILoggerRepository getTreeLoggerRepository();
-
public IAsterixAppRuntimeContextProvider getAsterixAppRuntimeContextProvider();
public String getId();
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
index 4b5eb9b..dd1e7b4 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
@@ -22,6 +22,7 @@
private static final long serialVersionUID = 2084227360840799662L;
+ public static final String lineSeparator = System.getProperty("line.separator");
public static final int LOG_MAGIC_NUMBER = 123456789;
public static final String LOG_DIR_SUFFIX = ".txnLogDir";
private static final String DEFAULT_LOG_FILE_PREFIX = "asterix_transaction_log";
@@ -35,14 +36,10 @@
private final int logPageSize;
// number of log pages in the log buffer.
private final int numLogPages;
- // time in milliseconds
- private final long groupCommitWaitPeriod;
// logBufferSize = logPageSize * numLogPages;
private final int logBufferSize;
// maximum size of each log file
private final long logPartitionSize;
- // default disk sector size
- private final int diskSectorSize;
public LogManagerProperties(AsterixTransactionProperties txnProperties, String nodeId) {
this.logDirKey = new String(nodeId + LOG_DIR_SUFFIX);
@@ -51,12 +48,9 @@
long logPartitionSize = txnProperties.getLogPartitionSize();
this.logDir = txnProperties.getLogDirectory(nodeId);
this.logFilePrefix = DEFAULT_LOG_FILE_PREFIX;
- this.groupCommitWaitPeriod = txnProperties.getGroupCommitInterval();
-
this.logBufferSize = logPageSize * numLogPages;
//make sure that the log partition size is the multiple of log buffer size.
this.logPartitionSize = (logPartitionSize / logBufferSize) * logBufferSize;
- this.diskSectorSize = txnProperties.getLogDiskSectorSize();
}
public long getLogPartitionSize() {
@@ -83,27 +77,17 @@
return logBufferSize;
}
- public long getGroupCommitWaitPeriod() {
- return groupCommitWaitPeriod;
- }
-
public String getLogDirKey() {
return logDirKey;
}
- public int getDiskSectorSize() {
- return diskSectorSize;
- }
-
public String toString() {
StringBuilder builder = new StringBuilder();
- builder.append("log_dir_ : " + logDir + FileUtil.lineSeparator);
- builder.append("log_file_prefix" + logFilePrefix + FileUtil.lineSeparator);
- builder.append("log_page_size : " + logPageSize + FileUtil.lineSeparator);
- builder.append("num_log_pages : " + numLogPages + FileUtil.lineSeparator);
- builder.append("log_partition_size : " + logPartitionSize + FileUtil.lineSeparator);
- builder.append("group_commit_wait_period : " + groupCommitWaitPeriod + FileUtil.lineSeparator);
- builder.append("disk_sector_size : " + diskSectorSize + FileUtil.lineSeparator);
+ builder.append("log_dir_ : " + logDir + lineSeparator);
+ builder.append("log_file_prefix" + logFilePrefix + lineSeparator);
+ builder.append("log_page_size : " + logPageSize + lineSeparator);
+ builder.append("num_log_pages : " + numLogPages + lineSeparator);
+ builder.append("log_partition_size : " + logPartitionSize + lineSeparator);
return builder.toString();
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java
deleted file mode 100644
index 58fc62a..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * A utility class providing helper methods for the {@link ILogManager}
- */
-public class LogUtil {
-
- private static final Logger LOGGER = Logger.getLogger(LogUtil.class.getName());
-
- // read the log directory and initialize log anchor to point to the
- // current log partition file and the offset where the log manager shall
- // continue to insert log records.
-
- public static PhysicalLogLocator initializeLogAnchor(ILogManager logManager) throws ACIDException {
- int fileId = 0;
- long offset = 0;
- LogManagerProperties logManagerProperties = logManager.getLogManagerProperties();
- File logDir = new File(logManagerProperties.getLogDir());
- try {
- if (logDir.exists()) {
- List<String> logFiles = getLogFiles(logManagerProperties);
- if (logFiles == null || logFiles.size() == 0) {
- FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
- }
- } else {
- File logFile = new File(LogUtil.getLogFilePath(logManagerProperties,
- Long.parseLong(logFiles.get(logFiles.size() - 1))));
- fileId = logFiles.size() - 1;
- offset = logFile.length();
- }
- } else {
- FileUtil.createNewDirectory(logManagerProperties.getLogDir());
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
- }
- FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
- }
- }
- } catch (IOException ioe) {
- throw new ACIDException("Unable to initialize log anchor", ioe);
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" file id :" + fileId + " offset " + offset);
- }
- return new PhysicalLogLocator(fileId, offset, logManager);
- }
-
- public static List<String> getLogFiles(final LogManagerProperties logManagerProperties) {
- File logDir = new File(logManagerProperties.getLogDir());
- String[] logFiles = new String[0];
- List<String> logFileCollection = new ArrayList<String>();
- if (logDir.exists()) {
- logFiles = logDir.list(new FilenameFilter() {
- public boolean accept(File dir, String name) {
- if (name.startsWith(logManagerProperties.getLogFilePrefix())) {
- return true;
- }
- return false;
- }
- });
- }
- for (String logFile : logFiles) {
- logFileCollection.add(logFile.substring(logManagerProperties.getLogFilePrefix().length() + 1));
- }
- Collections.sort(logFileCollection, new Comparator<String>() {
- @Override
- public int compare(String arg0, String arg1) {
- return Integer.parseInt(arg0) - Integer.parseInt(arg1);
- }
- });
- return logFileCollection;
- }
-
- public static long getFileId(String logFilePath, LogManagerProperties logManagerProperties) {
- String logFileName = logFilePath;
- if (logFilePath.contains(File.separator)) {
- logFileName = logFilePath.substring(logFilePath.lastIndexOf(File.separator));
- }
- return Long.parseLong(logFileName.substring(logFileName.indexOf(logManagerProperties.getLogFilePrefix())));
- }
-
- public static String getLogFilePath(LogManagerProperties logManagerProperties, long fileId) {
- return logManagerProperties.getLogDir() + File.separator + logManagerProperties.getLogFilePrefix() + "_"
- + fileId;
- }
-
- public static LogicalLogLocator getDummyLogicalLogLocator(ILogManager logManager) {
- LogicalLogLocator logicalLogLocator = new LogicalLogLocator(-1, null, -1, logManager);
- return logicalLogLocator;
- }
-
- /*
- * given a lsn, get the offset within the log file where the corresponding
- * log record is (to be) placed.
- */
- public static long getFileOffset(ILogManager logManager, long lsn) {
- return lsn % logManager.getLogManagerProperties().getLogPartitionSize();
- }
-
- /*
- * given a lsn, get the file id that contains the log record.
- */
- public static long getFileId(ILogManager logManager, long lsn) {
- return lsn / logManager.getLogManagerProperties().getLogPartitionSize();
- }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
deleted file mode 100644
index 9ba7682..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Represents a location of a log record. The location has two parts to it. A
- * LogicalLogLocator extends PhyscialLogLocator and hence can also be used to
- * determine the physical location of the log record on the local filesystem. In
- * addition to the physical location, a LogicalLogLocator also contains a handle
- * to an in-memory buffer and an offset within the buffer where the log record
- * resides.
- */
-public class LogicalLogLocator extends PhysicalLogLocator {
-
- private IBuffer buffer;
- private AtomicInteger memoryOffset;
-
- public LogicalLogLocator(long lsnValue, IBuffer buffer, int bufferOffset, ILogManager logManager) {
- super(lsnValue, logManager);
- this.buffer = buffer;
- this.memoryOffset = new AtomicInteger(bufferOffset);
-
- }
-
- public IBuffer getBuffer() {
- return buffer;
- }
-
- public void setBuffer(IBuffer buffer) {
- this.buffer = buffer;
- }
-
- public int getMemoryOffset() {
- return memoryOffset.get();
- }
-
- public void setMemoryOffset(int memoryOffset) {
- this.memoryOffset.set(memoryOffset);
- }
-
- @Override
- public String toString() {
- return super.toString() + " " + "memoryOffset:" + memoryOffset;
- }
-
- public boolean checkValidity() {
- return true;
- }
-
- public long increaseMemoryOffset(int delta) {
- return memoryOffset.addAndGet(delta);
- }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
similarity index 66%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
index fb865aa..4480aba 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
@@ -14,31 +14,39 @@
*/
package edu.uci.ics.asterix.common.transactions;
-public class MutableResourceId{
- long id;
+public class MutableLong {
+ private long val;
- public MutableResourceId(long id) {
- this.id = id;
+
+ public MutableLong() {
+ }
+
+ public MutableLong(long val) {
+ this.val = val;
+ }
+
+ public void set(long val) {
+ this.val = val;
}
- public void setId(long id) {
- this.id = id;
+ public long get() {
+ return val;
}
-
- public long getId() {
- return id;
- }
-
+
@Override
public int hashCode() {
- return (int)id;
+ return (int)val;
}
@Override
public boolean equals(Object o) {
- if ((o == null) || !(o instanceof MutableResourceId)) {
+ if (o == this) {
+ return true;
+ }
+ if (!(o instanceof MutableLong)) {
return false;
}
- return ((MutableResourceId) o).id == this.id;
+ return ((MutableLong) o).val == val;
}
+
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
deleted file mode 100644
index b83a3f3..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * Represents the physical location of a log record. The physical location i
- * deciphered from the contained lsn that is broken down into a file id and an
- * offset within the file. The mapping between fileId and the path on the local
- * file system is maintained by the log manager (@see ILogManager) path on the
- * local file system.
- */
-public class PhysicalLogLocator {
-
- // The log sequence number corresponding to the log record that is being
- // referred to.
- private final AtomicLong lsn;
-
- // Handle to the log manager that wrote the log record.
- private final ILogManager logManager;
-
- public static long getLsnValue(long fileId, long offset, ILogManager logManager) {
- return fileId * logManager.getLogManagerProperties().getLogPartitionSize() + offset;
- }
-
- public PhysicalLogLocator(long lsn, ILogManager logManager) {
- this.lsn = new AtomicLong(lsn);
- this.logManager = logManager;
- }
-
- public PhysicalLogLocator(long fileId, long offset, ILogManager logManager) {
- this.lsn = new AtomicLong(getLsnValue(fileId, offset, logManager));
- this.logManager = logManager;
- }
-
- @Override
- public String toString() {
- return "lsn :" + lsn.get();
- }
-
- public long getFileId() {
- return LogUtil.getFileId(logManager, lsn.get());
- }
-
- public boolean compareAndSet(long expect, long update) {
- return lsn.compareAndSet(expect, update);
- }
-
- public long getFileOffset() {
- return LogUtil.getFileOffset(logManager, lsn.get());
- }
-
- public long getLsn() {
- return lsn.get();
- }
-
- public long incrementLsn() {
- return lsn.incrementAndGet();
- }
-
- public long incrementLsn(long delta) {
- return lsn.addAndGet(delta);
- }
-
- public void setLsn(long lsn) {
- this.lsn.set(lsn);
- }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java
deleted file mode 100644
index b513fad..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.common.transactions;
-
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-
-/**
- * Represents the state of a transaction thread. The state contains information
- * that includes the tuple being operated, the operation and the location of the
- * log record corresponding to the operation.
- */
-public class ReusableLogContentObject {
-
- private LogicalLogLocator logicalLogLocator;
- private IndexOperation newOperation;
- private ITupleReference newValue;
- private IndexOperation oldOperation;
- private ITupleReference oldValue;
-
- public ReusableLogContentObject(LogicalLogLocator logicalLogLocator, IndexOperation newOperation,
- ITupleReference newValue, IndexOperation oldOperation, ITupleReference oldValue) {
- this.logicalLogLocator = logicalLogLocator;
- this.newOperation = newOperation;
- this.newValue = newValue;
- this.oldOperation = oldOperation;
- this.oldValue = oldValue;
- }
-
- public synchronized LogicalLogLocator getLogicalLogLocator() {
- return logicalLogLocator;
- }
-
- public synchronized void setLogicalLogLocator(LogicalLogLocator logicalLogLocator) {
- this.logicalLogLocator = logicalLogLocator;
- }
-
- public synchronized void setNewOperation(IndexOperation newOperation) {
- this.newOperation = newOperation;
- }
-
- public synchronized IndexOperation getNewOperation() {
- return newOperation;
- }
-
- public synchronized void setNewValue(ITupleReference newValue) {
- this.newValue = newValue;
- }
-
- public synchronized ITupleReference getNewValue() {
- return newValue;
- }
-
- public synchronized void setOldOperation(IndexOperation oldOperation) {
- this.oldOperation = oldOperation;
- }
-
- public synchronized IndexOperation getOldOperation() {
- return oldOperation;
- }
-
- public synchronized void setOldValue(ITupleReference oldValue) {
- this.oldValue = oldValue;
- }
-
- public synchronized ITupleReference getOldValue() {
- return oldValue;
- }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
deleted file mode 100644
index 64c90f4..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.common.transactions;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Represents a repository containing Resource Managers and Resources in the
- * transaction eco-system. Operations on a resource require acquiring
- * appropriate locks (for isolation) and writing logs (durability). Every
- * resource is managed by an associated resource manager that contains the logic
- * to interpret the logs and take necessary action(s) during roll back or
- * recovery. An example of resource is a @see ITreeIndex that is managed by a
- * resource manager @see TreeResourceManager
- */
-public class TransactionalResourceManagerRepository {
-
- private Map<Byte, IResourceManager> resourceMgrRepository = new HashMap<Byte, IResourceManager>(); // repository
-
- public void registerTransactionalResourceManager(byte id, IResourceManager resourceMgr) {
- synchronized (resourceMgrRepository) {
- if (resourceMgrRepository.get(id) == null) {
- resourceMgrRepository.put(id, resourceMgr);
- }
- }
- }
-
- public IResourceManager getTransactionalResourceMgr(byte id) {
- synchronized (resourceMgrRepository) {
- return resourceMgrRepository.get(id);
- }
- }
-}
diff --git a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
index a807e39..d41815d 100644
--- a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
+++ b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
@@ -325,6 +325,8 @@
List<CompilationUnit> cUnits = testCaseCtx.getTestCase().getCompilationUnit();
for (CompilationUnit cUnit : cUnits) {
+ LOGGER.info("Starting [TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/"
+ + cUnit.getName() + " ... ");
testFileCtxs = testCaseCtx.getTestFiles(cUnit);
expectedResultFileCtxs = testCaseCtx.getExpectedResultFiles(cUnit);
diff --git a/asterix-doc/src/site/markdown/install.md b/asterix-doc/src/site/markdown/install.md
index 4d53f75..9ba8ea3 100644
--- a/asterix-doc/src/site/markdown/install.md
+++ b/asterix-doc/src/site/markdown/install.md
@@ -880,3 +880,32 @@
##### Answer #####
It is recommended that MANAGIX_HOME is not on the NFS. Managix produces artifacts/logs on disk which are not required to be shared.
As such an overhead in creating the artifacts/logs on the NFS should be avoided.
+
+##### Question #####
+
+Question: How do we change the underlying code (apply a code patch) for an 'active' asterix instance?
+
+##### Answer #####
+
+At times, end-user (particularly asterix developer) may run into the need to altering the underlying code that is being run by an asterix instance. In the current version of managix, this can be achieved as follows:-
+
+Assume that you have an 'active' instance by the name a1 that is running version v1 of asterix.
+You have a revised version of asterix - v2 that fixes some bug(s).
+
+To upgrade asterix from v1 to v2:-
+
+step 1) managix stop -n a1
+
+step 2) managix shutdown
+
+step 3) copy asterix-server zip (version v2) to $MANAGIX_HOME/asterix/
+
+step 4) managix start -n a1
+
+a1 now is running on version v2.
+
+Limitations:-
+
+a) Obviously this wont work in a situation where v2 has made a change that is incompatible with earlier version, such altering schema.
+
+b) A change in asterix zip applies to all existing instances (after a restart) and subsequent instances that user creates.
diff --git a/asterix-doc/src/site/site.xml b/asterix-doc/src/site/site.xml
index 8c2a8a8..150544e 100644
--- a/asterix-doc/src/site/site.xml
+++ b/asterix-doc/src/site/site.xml
@@ -59,7 +59,7 @@
</links>
<menu name="Documentation">
- <item name="Installing AsterixDB using Managix" href="install.html"/>
+ <item name="Installing and Managing AsterixDB using Managix" href="install.html"/>
<item name="AsterixDB 101: An ADM and AQL Primer" href="aql/primer.html"/>
<item name="Asterix Data Model (ADM)" href="aql/datamodel.html"/>
<item name="Asterix Functions" href="aql/functions.html"/>
diff --git a/asterix-installer/src/main/resources/conf/asterix-configuration.xml b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
index e542efd..084a3f8 100644
--- a/asterix-installer/src/main/resources/conf/asterix-configuration.xml
+++ b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
@@ -68,6 +68,14 @@
</description>
</property>
+ <property>
+ <name>storage.memorycomponent.numcomponents</name>
+ <value>2</value>
+ <description>The number of memory components to be used per lsm index.
+ (Default = 2)
+ </description>
+ </property>
+
<property>
<name>storage.memorycomponent.globalbudget</name>
<value>536870192</value>
@@ -118,21 +126,6 @@
</property>
<property>
- <name>txn.log.disksectorsize</name>
- <value>4096</value>
- <description>The size of a disk sector. (Default = "4096")
- </description>
- </property>
-
- <property>
- <name>txn.log.groupcommitinterval</name>
- <value>1</value>
- <description>The group commit wait time in milliseconds. (Default =
- "10" // 0.1ms)
- </description>
- </property>
-
- <property>
<name>txn.log.checkpoint.lsnthreshold</name>
<value>67108864</value>
<description>The size of the window that the maximum LSN is allowed to
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index 8a7f6e1..c6d83cf 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
@@ -32,9 +32,8 @@
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.asterix.common.transactions.DatasetId;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -90,7 +89,6 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
-import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
@@ -99,10 +97,12 @@
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.exceptions.TreeIndexDuplicateKeyException;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
public class MetadataNode implements IMetadataNode {
private static final long serialVersionUID = 1L;
@@ -125,19 +125,21 @@
@Override
public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
- transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
+ txnCtx.setMetadataTransaction(true);
}
@Override
public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
}
@Override
public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
try {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId,
+ false);
transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
} catch (ACIDException e) {
e.printStackTrace();
@@ -147,13 +149,13 @@
@Override
public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
transactionSubsystem.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
}
@Override
public void unlock(JobId jobId) throws ACIDException, RemoteException {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
}
@@ -163,7 +165,7 @@
DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
- } catch (BTreeDuplicateKeyException e) {
+ } catch (TreeIndexDuplicateKeyException e) {
throw new MetadataException("A dataverse with this name " + dataverse.getDataverseName()
+ " already exists.", e);
} catch (Exception e) {
@@ -193,7 +195,7 @@
ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
dataset.getDatasetName());
insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
- } catch (BTreeDuplicateKeyException e) {
+ } catch (TreeIndexDuplicateKeyException e) {
throw new MetadataException("A dataset with this name " + dataset.getDatasetName()
+ " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
} catch (Exception e) {
@@ -207,7 +209,7 @@
IndexTupleTranslator tupleWriter = new IndexTupleTranslator(true);
ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
- } catch (BTreeDuplicateKeyException e) {
+ } catch (TreeIndexDuplicateKeyException e) {
throw new MetadataException("An index with name '" + index.getIndexName() + "' already exists.", e);
} catch (Exception e) {
throw new MetadataException(e);
@@ -220,7 +222,7 @@
NodeTupleTranslator tupleReaderWriter = new NodeTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
- } catch (BTreeDuplicateKeyException e) {
+ } catch (TreeIndexDuplicateKeyException e) {
throw new MetadataException("A node with name '" + node.getNodeName() + "' already exists.", e);
} catch (Exception e) {
throw new MetadataException(e);
@@ -233,7 +235,7 @@
NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
- } catch (BTreeDuplicateKeyException e) {
+ } catch (TreeIndexDuplicateKeyException e) {
throw new MetadataException("A nodegroup with name '" + nodeGroup.getNodeGroupName() + "' already exists.",
e);
} catch (Exception e) {
@@ -247,7 +249,7 @@
DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, true);
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
- } catch (BTreeDuplicateKeyException e) {
+ } catch (TreeIndexDuplicateKeyException e) {
throw new MetadataException("A datatype with name '" + datatype.getDatatypeName() + "' already exists.", e);
} catch (Exception e) {
throw new MetadataException(e);
@@ -262,7 +264,7 @@
ITupleReference functionTuple = tupleReaderWriter.getTupleFromMetadataEntity(function);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
- } catch (BTreeDuplicateKeyException e) {
+ } catch (TreeIndexDuplicateKeyException e) {
throw new MetadataException("A function with this name " + function.getName() + " and arity "
+ function.getArity() + " already exists in dataverse '" + function.getDataverseName() + "'.", e);
} catch (Exception e) {
@@ -286,21 +288,22 @@
IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
lsmIndex, IndexOperation.INSERT);
- IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+ ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- txnCtx.setTransactionType(TransactionType.READ_WRITE);
- txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+ txnCtx.setWriteTxn(true);
+ txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
+ metadataIndex.isPrimaryIndex());
// TODO: fix exceptions once new BTree exception model is in hyracks.
- indexAccessor.insert(tuple);
+ indexAccessor.forceInsert(tuple);
indexLifecycleManager.close(resourceID);
}
private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws Exception {
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
if (metadataIndex.isPrimaryIndex()) {
return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
@@ -607,13 +610,14 @@
// prepare a Callback for logging
IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
lsmIndex, IndexOperation.DELETE);
- IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+ ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
- ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
- txnCtx.setTransactionType(TransactionType.READ_WRITE);
- txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
+ ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+ txnCtx.setWriteTxn(true);
+ txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
+ metadataIndex.isPrimaryIndex());
- indexAccessor.delete(tuple);
+ indexAccessor.forceDelete(tuple);
indexLifecycleManager.close(resourceID);
}
@@ -1116,7 +1120,7 @@
ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
- } catch (BTreeDuplicateKeyException e) {
+ } catch (TreeIndexDuplicateKeyException e) {
throw new MetadataException("A adapter with this name " + adapter.getAdapterIdentifier().getAdapterName()
+ " already exists in dataverse '" + adapter.getAdapterIdentifier().getNamespace() + "'.", e);
} catch (Exception e) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index 2d60c41..99d2e21 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -33,10 +33,13 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
+import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
+import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
import edu.uci.ics.asterix.metadata.IDatasetDetails;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -63,8 +66,6 @@
import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -96,6 +97,7 @@
*/
public class MetadataBootstrap {
private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
+ public static final boolean IS_DEBUG_MODE = false;//true
private static IAsterixAppRuntimeContext runtimeContext;
@@ -142,15 +144,6 @@
MetadataSecondaryIndexes.init();
initLocalIndexArrays();
- TransactionalResourceManagerRepository resourceRepository = runtimeContext.getTransactionSubsystem()
- .getTransactionalResourceRepository();
- resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_BTREE, new IndexResourceManager(
- ResourceType.LSM_BTREE, runtimeContext.getTransactionSubsystem()));
- resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_RTREE, new IndexResourceManager(
- ResourceType.LSM_RTREE, runtimeContext.getTransactionSubsystem()));
- resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_INVERTED_INDEX,
- new IndexResourceManager(ResourceType.LSM_INVERTED_INDEX, runtimeContext.getTransactionSubsystem()));
-
AsterixMetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
metadataNodeName = metadataProperties.getMetadataNodeName();
metadataStore = metadataProperties.getMetadataStore();
@@ -206,6 +199,9 @@
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
} catch (Exception e) {
try {
+ if (IS_DEBUG_MODE) {
+ e.printStackTrace();
+ }
MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
} catch (Exception e2) {
e.addSuppressed(e2);
@@ -348,20 +344,21 @@
+ IndexFileNameUtil.prepareFileName(metadataStore + File.separator + index.getFileNameRelativePath(),
runtimeContext.getMetaDataIODeviceId());
FileReference file = new FileReference(new File(filePath));
- IVirtualBufferCache virtualBufferCache = runtimeContext.getVirtualBufferCache(index.getDatasetId().getId());
+ List<IVirtualBufferCache> virtualBufferCaches = runtimeContext.getVirtualBufferCaches(index.getDatasetId()
+ .getId());
ITypeTraits[] typeTraits = index.getTypeTraits();
IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
LSMBTree lsmBtree = null;
long resourceID = -1;
- AsterixRuntimeComponentsProvider rtcProvider = index.isPrimaryIndex() ? AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER
- : AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER;
ILSMOperationTracker opTracker = index.isPrimaryIndex() ? runtimeContext.getLSMBTreeOperationTracker(index
- .getDatasetId().getId()) : new BaseOperationTracker(LSMBTreeIOOperationCallbackFactory.INSTANCE);
+ .getDatasetId().getId()) : new BaseOperationTracker((DatasetLifecycleManager) indexLifecycleManager,
+ index.getDatasetId().getId());
if (create) {
- lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, bufferCache, fileMapProvider, typeTraits,
+ lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, bufferCache, fileMapProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, runtimeContext.getBloomFilterFalsePositiveRate(),
- runtimeContext.getLSMMergePolicy(), opTracker, runtimeContext.getLSMIOScheduler(), rtcProvider);
+ runtimeContext.getLSMMergePolicy(), opTracker, runtimeContext.getLSMIOScheduler(),
+ LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback());
lsmBtree.create();
resourceID = runtimeContext.getResourceIdFactory().createId();
ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
@@ -376,11 +373,11 @@
resourceID = localResourceRepository.getResourceByName(file.getFile().getPath()).getResourceId();
lsmBtree = (LSMBTree) indexLifecycleManager.getIndex(resourceID);
if (lsmBtree == null) {
- lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, bufferCache, fileMapProvider,
+ lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, bufferCache, fileMapProvider,
typeTraits, comparatorFactories, bloomFilterKeyFields,
runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(),
opTracker, runtimeContext.getLSMIOScheduler(),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER);
+ LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback());
indexLifecycleManager.register(resourceID, lsmBtree);
}
}
@@ -457,6 +454,9 @@
}
} catch (Exception e) {
try {
+ if (IS_DEBUG_MODE) {
+ e.printStackTrace();
+ }
MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
} catch (Exception e2) {
e.addSuppressed(e2);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index 5af996f..cf0c3d5 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -33,8 +33,11 @@
import edu.uci.ics.asterix.common.dataflow.AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor;
import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.parse.IParseFileSplitsDecl;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
import edu.uci.ics.asterix.formats.base.IDataFormat;
@@ -86,6 +89,7 @@
import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexSearchOperationCallbackFactory;
import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
@@ -546,17 +550,16 @@
primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
}
}
- AsterixRuntimeComponentsProvider rtcProvider = isSecondary ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
- : AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER;
+ AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), rtcProvider,
- isSecondary ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
+ isSecondary ? new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId())
: new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
- rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
- searchCallbackFactory);
+ LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
@@ -621,12 +624,12 @@
typeTraits, comparatorFactories, keyFields, new LSMRTreeDataflowHelperFactory(
valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
- nestedKeyType.getTypeTag(), comparatorFactories.length),
- storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, proposeLinearizer(nestedKeyType.getTypeTag(),
+ comparatorFactories.length), storageProperties.getBloomFilterFalsePositiveRate()),
+ retainInput, searchCallbackFactory);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
} catch (MetadataException me) {
@@ -779,10 +782,10 @@
splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, numElementsHint, true,
new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
splitsAndConstraint.second);
@@ -847,10 +850,10 @@
appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
.getBloomFilterFalsePositiveRate()), null, modificationCallbackFactory, true);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
@@ -1043,10 +1046,10 @@
appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, storageProperties
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
.getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory,
false);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
@@ -1171,10 +1174,10 @@
appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation, indexOp,
new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER, storageProperties
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, storageProperties
.getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
splitsAndConstraint.second);
@@ -1265,13 +1268,12 @@
comparatorFactories, null, fieldPermutation, indexOp, new LSMRTreeDataflowHelperFactory(
valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
- AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
- nestedKeyType.getTypeTag(), comparatorFactories.length),
- storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
- modificationCallbackFactory, false);
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, proposeLinearizer(nestedKeyType.getTypeTag(),
+ comparatorFactories.length), storageProperties.getBloomFilterFalsePositiveRate()),
+ filterFactory, modificationCallbackFactory, false);
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
} catch (MetadataException | IOException e) {
throw new AlgebricksException(e);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index 6c55f12..541d703 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -55,7 +55,7 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
+import edu.uci.ics.hyracks.storage.am.common.exceptions.TreeIndexDuplicateKeyException;
/**
* Translates a Datatype metadata entity to an ITupleReference and vice versa.
@@ -437,7 +437,7 @@
mn.insertIntoDatatypeSecondaryIndex(jobId, topLevelType.getDataverseName(), typeName,
topLevelType.getDatatypeName());
- } catch (BTreeDuplicateKeyException e) {
+ } catch (TreeIndexDuplicateKeyException e) {
// The key may have been inserted by a previous DDL statement or by
// a previous nested type.
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
index b016af0..8471056 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
@@ -31,7 +31,7 @@
private static final long serialVersionUID = 1L;
- public static final ADayTimeDurationSerializerDeserializer INSTNACE = new ADayTimeDurationSerializerDeserializer();
+ public static final ADayTimeDurationSerializerDeserializer INSTANCE = new ADayTimeDurationSerializerDeserializer();
@SuppressWarnings("unchecked")
private static final ISerializerDeserializer<ADayTimeDuration> dayTimeDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -56,7 +56,7 @@
}
}
- public static void parse(String durationString, DataOutput out) throws HyracksDataException {
+ public void parse(String durationString, DataOutput out) throws HyracksDataException {
try {
ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aDayTimeDuration,
ADurationParseOption.All);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
index 486f271..14ec491 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
@@ -101,10 +101,10 @@
return ADurationSerializerDeserializer.INSTANCE.deserialize(in);
}
case YEARMONTHDURATION: {
- return AYearMonthDurationerializerDeserializer.INSTANCE.deserialize(in);
+ return AYearMonthDurationSerializerDeserializer.INSTANCE.deserialize(in);
}
case DAYTIMEDURATION: {
- return ADayTimeDurationSerializerDeserializer.INSTNACE.deserialize(in);
+ return ADayTimeDurationSerializerDeserializer.INSTANCE.deserialize(in);
}
case INTERVAL: {
return AIntervalSerializerDeserializer.INSTANCE.deserialize(in);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AYearMonthDurationerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
similarity index 89%
rename from asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AYearMonthDurationerializerDeserializer.java
rename to asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
index 79bf3c0..3cd6e66 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AYearMonthDurationerializerDeserializer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
@@ -27,11 +27,11 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-public class AYearMonthDurationerializerDeserializer implements ISerializerDeserializer<AYearMonthDuration> {
+public class AYearMonthDurationSerializerDeserializer implements ISerializerDeserializer<AYearMonthDuration> {
private static final long serialVersionUID = 1L;
- public static final AYearMonthDurationerializerDeserializer INSTANCE = new AYearMonthDurationerializerDeserializer();
+ public static final AYearMonthDurationSerializerDeserializer INSTANCE = new AYearMonthDurationSerializerDeserializer();
@SuppressWarnings("unchecked")
private static final ISerializerDeserializer<AYearMonthDuration> yearMonthDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
@@ -56,7 +56,7 @@
}
}
- public static void parse(String durationString, DataOutput out) throws HyracksDataException {
+ public void parse(String durationString, DataOutput out) throws HyracksDataException {
try {
ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aYearMonthDuration,
ADurationParseOption.All);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
index ff1a14c..36f94a6 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
@@ -44,7 +44,7 @@
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
import edu.uci.ics.asterix.om.base.IAObject;
import edu.uci.ics.asterix.om.types.AOrderedListType;
@@ -129,10 +129,10 @@
return ADurationSerializerDeserializer.INSTANCE;
}
case YEARMONTHDURATION: {
- return AYearMonthDurationerializerDeserializer.INSTANCE;
+ return AYearMonthDurationSerializerDeserializer.INSTANCE;
}
case DAYTIMEDURATION: {
- return ADayTimeDurationSerializerDeserializer.INSTNACE;
+ return ADayTimeDurationSerializerDeserializer.INSTANCE;
}
case INTERVAL: {
return AIntervalSerializerDeserializer.INSTANCE;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ATimeParserFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ATimeParserFactory.java
index 2200ea3..3a0a80a 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ATimeParserFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ATimeParserFactory.java
@@ -181,11 +181,6 @@
}
}
- if (timezoneHour < GregorianCalendarSystem.TIMEZONE_HOUR_MIN
- || timezoneHour > GregorianCalendarSystem.TIMEZONE_HOUR_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone hour " + timezoneHour);
- }
-
int temp_offset = (timeString.charAt(start + 3) == ':') ? 1 : 0;
for (int i = 0; i < 2; i++) {
@@ -197,15 +192,11 @@
}
}
- if (timezoneMinute < GregorianCalendarSystem.TIMEZONE_MIN_MIN
- || timezoneMinute > GregorianCalendarSystem.TIMEZONE_MIN_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone minute " + timezoneMinute);
- }
+ timezone = (int) (timezoneHour * GregorianCalendarSystem.CHRONON_OF_HOUR + timezoneMinute
+ * GregorianCalendarSystem.CHRONON_OF_MINUTE);
- if (timeString.charAt(start) == '-') {
- timezone = (byte) -((timezoneHour * 4) + timezoneMinute / 15);
- } else {
- timezone = (byte) ((timezoneHour * 4) + timezoneMinute / 15);
+ if (timeString.charAt(start) == '+') {
+ timezone *= -1;
}
}
return timezone;
@@ -345,11 +336,6 @@
}
}
- if (timezoneHour < GregorianCalendarSystem.TIMEZONE_HOUR_MIN
- || timezoneHour > GregorianCalendarSystem.TIMEZONE_HOUR_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone hour " + timezoneHour);
- }
-
int temp_offset = (timeString[start + 3] == ':') ? 1 : 0;
for (int i = 0; i < 2; i++) {
@@ -360,15 +346,11 @@
}
}
- if (timezoneMinute < GregorianCalendarSystem.TIMEZONE_MIN_MIN
- || timezoneMinute > GregorianCalendarSystem.TIMEZONE_MIN_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone minute " + timezoneMinute);
- }
+ timezone = (int) (timezoneHour * GregorianCalendarSystem.CHRONON_OF_HOUR + timezoneMinute
+ * GregorianCalendarSystem.CHRONON_OF_MINUTE);
- if (timeString[start] == '-') {
- timezone = (byte) -((timezoneHour * 4) + timezoneMinute / 15);
- } else {
- timezone = (byte) ((timezoneHour * 4) + timezoneMinute / 15);
+ if (timeString[start] == '+') {
+ timezone *= -1;
}
}
return timezone;
@@ -508,11 +490,6 @@
}
}
- if (timezoneHour < GregorianCalendarSystem.TIMEZONE_HOUR_MIN
- || timezoneHour > GregorianCalendarSystem.TIMEZONE_HOUR_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone hour " + timezoneHour);
- }
-
int temp_offset = (timeString[start + 3] == ':') ? 1 : 0;
for (int i = 0; i < 2; i++) {
@@ -523,15 +500,11 @@
}
}
- if (timezoneMinute < GregorianCalendarSystem.TIMEZONE_MIN_MIN
- || timezoneMinute > GregorianCalendarSystem.TIMEZONE_MIN_MAX) {
- throw new HyracksDataException(timeErrorMessage + ": time zone minute " + timezoneMinute);
- }
+ timezone = (int) (timezoneHour * GregorianCalendarSystem.CHRONON_OF_HOUR + timezoneMinute
+ * GregorianCalendarSystem.CHRONON_OF_MINUTE);
- if (timeString[start] == '-') {
- timezone = (byte) -((timezoneHour * 4) + timezoneMinute / 15);
- } else {
- timezone = (byte) ((timezoneHour * 4) + timezoneMinute / 15);
+ if (timeString[start] == '+') {
+ timezone *= -1;
}
}
return timezone;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/AsterixTemporalTypeParseException.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/AsterixTemporalTypeParseException.java
new file mode 100644
index 0000000..9354d5f
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/AsterixTemporalTypeParseException.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.base.temporal;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class AsterixTemporalTypeParseException extends HyracksDataException {
+
+ private static final long serialVersionUID = 1L;
+
+ public AsterixTemporalTypeParseException() {
+ super();
+ }
+
+ public AsterixTemporalTypeParseException(Exception ex) {
+ super(ex);
+ }
+
+ public AsterixTemporalTypeParseException(String msg) {
+ super(msg);
+ }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/DateTimeFormatUtils.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/DateTimeFormatUtils.java
new file mode 100644
index 0000000..f0ee739
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/DateTimeFormatUtils.java
@@ -0,0 +1,965 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.base.temporal;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.TimeZone;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * {@link DateTimeFormatUtils} provides the utility methods to parse and print a date/time/datetime
+ * value based on the given format string. The format string may contain the following <b>format characters</b> (note that
+ * format string is <b>case-sensitive</b>):
+ * <p/>
+ * - <b>Y</b>: a digit for the year field. At most 4 year format characters are allowed for a valid format string.<br/>
+ * - <b>M</b>: a digit or character for the month field. At most 3 month format characters are allowed for a valid format string. When three month format characters are used, the shorten month names (like JAN, FEB etc.) are expected in the string to be parsed. Otherwise digits are expected.<br/>
+ * - <b>D</b>: a digit for the day field. At most 2 day format characters are allowed.<br/>
+ * - <b>h</b>: a digit for the hour field. At most 2 hour format characters are allowed.<br/>
+ * - <b>m</b>: a digit for the minute field. At most 2 minute format characters are allowed.<br/>
+ * - <b>s</b>: a digit for the second field. At most 2 second format characters are allowed.<br/>
+ * - <b>n</b>: a digit for the millisecond field. At most 3 millisecond format characters are allowed.<br/>
+ * - <b>a</b>: the AM/PM field. At most 1 am/pm format character is allowed, and it matches with AM and PM case-insensitively. <br/>
+ * - <b>z</b>: the timezone field. At most 1 timezone format characters are allowed. The valid timezone string matching with this format character include:<br/>
+ * -- <b>Z</b>: a single upper-case character representing the UTC timezone;<br/>
+ * -- <b>[UTC|GMT]+xx[:]xx</b>: representing a timezone by providing the actual offset time from the UTC time;<br/>
+ * -- A string representation of a timezone like PST, Asia/Shanghai. The names of the timezones are following the Zoneinfo database provided by the JDK library. See {@link TimeZone} for more details on this.<br/>
+ * - <b>Separators</b>: separators that can be used to separate the different fields. Currently only the following characters can be used as separator: <b>-(hyphen), :(colon), /(solidus), .(period) and ,(comma)</b>.
+ * <p/>
+ * For the matching algorithm, both the format string and the data string are scanned from the beginning to the end, and the algorithm tried to match the format with the characters/digits/separators in the data string. The format string represents the <b>minimum</b> length of the required field (similar to the C-style printf formatting). This means that something like a year <it>1990</it> will match with the format strings <it>Y, YY, YYY and YYYY</it>.
+ * <p/>
+ * If the given string cannot be parsed by the given format string, an {@link AsterixTemporalTypeParseException} will be returned.
+ */
+public class DateTimeFormatUtils {
+
+ private final GregorianCalendarSystem CAL = GregorianCalendarSystem.getInstance();
+
+ // For time
+ private final char HOUR_CHAR = 'h';
+ private final char MINUTE_CHAR = 'm';
+ private final char SECOND_CHAR = 's';
+ private final char MILLISECOND_CHAR = 'n';
+ private final char AMPM_CHAR = 'a';
+ private final char TIMEZONE_CHAR = 'z';
+
+ private final int MAX_HOUR_CHARS = 2;
+ private final int MAX_MINUTE_CHARS = 2;
+ private final int MAX_SECOND_CHARS = 2;
+ private final int MAX_MILLISECOND_CHARS = 3;
+ private final int MAX_AMPM_CHARS = 1;
+ private final int MAX_TIMEZONE_CHARS = 1;
+
+ private enum DateTimeProcessState {
+ INIT,
+ YEAR,
+ MONTH,
+ DAY,
+ WEEKDAY,
+ HOUR,
+ MINUTE,
+ SECOND,
+ MILLISECOND,
+ AMPM,
+ TIMEZONE,
+ SKIPPER,
+ SEPARATOR
+ }
+
+ // For date
+ private final char YEAR_CHAR = 'Y';
+ private final char MONTH_CHAR = 'M';
+ private final char DAY_CHAR = 'D';
+ private final char WEEKDAY_CHAR = 'W';
+
+ private final int MAX_YEAR_CHARS = 4;
+ private final int MAX_MONTH_CHARS = 3;
+ private final int MAX_DAY_CHARS = 2;
+ private final int MAX_WEEKDAY_CHAR = 1;
+
+ private final byte[][] MONTH_NAMES = new byte[][] { "jan".getBytes(), "feb".getBytes(), "mar".getBytes(),
+ "apr".getBytes(), "may".getBytes(), "jun".getBytes(), "jul".getBytes(), "aug".getBytes(), "sep".getBytes(),
+ "oct".getBytes(), "nov".getBytes(), "dec".getBytes() };
+
+ private final byte[][] WEEKDAY_FULL_NAMES = new byte[][] { "monday".getBytes(), "tuesday".getBytes(),
+ "wednesday".getBytes(), "thursday".getBytes(), "friday".getBytes(), "saturday".getBytes(),
+ "sunday".getBytes() };
+
+ private final byte[] UTC_BYTEARRAY = "utc".getBytes();
+ private final byte[] GMT_BYTEARRAY = "gmt".getBytes();
+
+ private final byte[] AM_BYTEARRAY = "am".getBytes();
+ private final byte[] PM_BYTEARRAY = "pm".getBytes();
+
+ // Separators, for both time and date
+ private final char HYPHEN_CHAR = '-';
+ private final char COLON_CHAR = ':';
+ private final char SOLIDUS_CHAR = '/';
+ private final char PERIOD_CHAR = '.';
+ private final char COMMA_CHAR = ',';
+ private final char T_CHAR = 'T';
+
+ // Skipper, representing a field with characters and numbers that to be skipped
+ private final char SKIPPER_CHAR = 'O';
+ private final int MAX_SKIPPER_CHAR = 1;
+
+ private final int MS_PER_MINUTE = 60 * 1000;
+ private final int MS_PER_HOUR = 60 * MS_PER_MINUTE;
+
+ private final byte TO_LOWER_OFFSET = 'A' - 'a';
+
+ private final String[] TZ_IDS = TimeZone.getAvailableIDs();
+ private final byte[][] TIMEZONE_IDS = new byte[TZ_IDS.length][];
+ {
+ Arrays.sort(TZ_IDS);
+ for (int i = 0; i < TIMEZONE_IDS.length; i++) {
+ TIMEZONE_IDS[i] = TZ_IDS[i].getBytes();
+ }
+ }
+
+ private final int[] TIMEZONE_OFFSETS = new int[TIMEZONE_IDS.length];
+ {
+ for (int i = 0; i < TIMEZONE_IDS.length; i++) {
+ TIMEZONE_OFFSETS[i] = TimeZone.getTimeZone(TZ_IDS[i]).getRawOffset();
+ }
+ }
+
+ private static class DateTimeFormatUtilsHolder {
+ private static final DateTimeFormatUtils INSTANCE = new DateTimeFormatUtils();
+ }
+
+ public static DateTimeFormatUtils getInstance() {
+ return DateTimeFormatUtilsHolder.INSTANCE;
+ }
+
+ private DateTimeFormatUtils() {
+ }
+
+ private int parseFormatField(byte[] format, int formatStart, int formatLength, int formatPointer, char formatChar,
+ int maxAllowedFormatCharCopied) {
+
+ int formatCharCopies = 0;
+
+ formatPointer++;
+ formatCharCopies++;
+ while (formatPointer < formatLength && format[formatStart + formatPointer] == formatChar) {
+ formatPointer++;
+ formatCharCopies++;
+ }
+ if (formatCharCopies > maxAllowedFormatCharCopied) {
+ throw new IllegalStateException("The format string for " + formatChar
+ + " is too long: expected no more than " + maxAllowedFormatCharCopied + " but got "
+ + formatCharCopies);
+ }
+
+ return formatCharCopies;
+ }
+
+ public enum DateTimeParseMode {
+ DATE_ONLY,
+ TIME_ONLY,
+ DATETIME
+ }
+
+ private boolean byteArrayEqualToString(byte[] barray, int start, int length, byte[] str) {
+ if (length != str.length) {
+ return false;
+ } else {
+ return byteArrayBeingWithString(barray, start, length, str);
+ }
+ }
+
+ private boolean byteArrayBeingWithString(byte[] barray, int start, int length, byte[] str) {
+ boolean beginWith = true;
+ if (length <= str.length) {
+ for (int i = 0; i < length; i++) {
+ if (toLower(barray[start + i]) != str[i]) {
+ beginWith = false;
+ break;
+ }
+ }
+ } else {
+ beginWith = false;
+ }
+ return beginWith;
+ }
+
+ private Comparator<byte[]> byteArrayComparator = new Comparator<byte[]>() {
+ @Override
+ public int compare(byte[] o1, byte[] o2) {
+ int i = 0;
+ for (; i < o1.length && i < o2.length; i++) {
+ if (o1[i] != o2[i]) {
+ return o1[i] - o2[i];
+ }
+ }
+ if (i < o1.length) {
+ return -1;
+ } else if (i < o2.length) {
+ return 1;
+ }
+ return 0;
+ }
+ };
+
+ private int monthIDSearch(byte[] barray, int start, int length) {
+ for (int i = 0; i < MONTH_NAMES.length; i++) {
+ if (byteArrayEqualToString(barray, start, length, MONTH_NAMES[i])) {
+ return i;
+ }
+ }
+ return -1;
+ }
+
+ private int weekdayIDSearch(byte[] barray, int start, int length) {
+ for (int i = 0; i < WEEKDAY_FULL_NAMES.length; i++) {
+ if (byteArrayBeingWithString(barray, start, length, WEEKDAY_FULL_NAMES[i])) {
+ return i;
+ }
+ }
+ return -1;
+ }
+
+ private int binaryTimezoneIDSearch(byte[] barray, int start, int length) {
+ return Arrays.binarySearch(TIMEZONE_IDS, 0, TIMEZONE_IDS.length,
+ Arrays.copyOfRange(barray, start, start + length), byteArrayComparator);
+ }
+
+ private int indexOf(byte[] barray, int start, int length, char c) {
+ int i = 0;
+ for (; i < length; i++) {
+ if (barray[start + i] == c) {
+ return i;
+ }
+ }
+ return -1;
+ }
+
+ private byte toLower(byte b) {
+ if (b >= 'A' && b <= 'Z') {
+ return (byte) (b - TO_LOWER_OFFSET);
+ }
+ return b;
+ }
+
+ private byte toUpper(byte b) {
+ if (b >= 'a' && b <= 'z') {
+ return (byte) (b + TO_LOWER_OFFSET);
+ }
+ return b;
+ }
+
+ public long parseDateTime(byte[] data, int dataStart, int dataLength, byte[] format, int formatStart,
+ int formatLength, DateTimeParseMode parseMode) throws AsterixTemporalTypeParseException {
+ int year = 0, month = 0, day = 0, hour = 0, min = 0, sec = 0, ms = 0, timezone = 0;
+
+ boolean negativeYear = false;
+ int formatCharCopies = 0;
+
+ int dataStringPointer = 0, formatPointer = 0;
+
+ byte separatorChar = '\0';
+
+ DateTimeProcessState processState = DateTimeProcessState.INIT;
+
+ int pointerMove = 0;
+
+ while (dataStringPointer < dataLength && formatPointer < formatLength) {
+ formatCharCopies = 0;
+ switch (format[formatStart + formatPointer]) {
+ case YEAR_CHAR:
+ processState = DateTimeProcessState.YEAR;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, YEAR_CHAR,
+ MAX_YEAR_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MONTH_CHAR:
+ processState = DateTimeProcessState.MONTH;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MONTH_CHAR,
+ MAX_MONTH_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case DAY_CHAR:
+ processState = DateTimeProcessState.DAY;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, DAY_CHAR,
+ MAX_DAY_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case WEEKDAY_CHAR:
+ processState = DateTimeProcessState.WEEKDAY;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, WEEKDAY_CHAR,
+ MAX_WEEKDAY_CHAR);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case HOUR_CHAR:
+ processState = DateTimeProcessState.HOUR;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, HOUR_CHAR,
+ MAX_HOUR_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MINUTE_CHAR:
+ processState = DateTimeProcessState.MINUTE;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MINUTE_CHAR,
+ MAX_MINUTE_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case SECOND_CHAR:
+ processState = DateTimeProcessState.SECOND;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, SECOND_CHAR,
+ MAX_SECOND_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MILLISECOND_CHAR:
+ processState = DateTimeProcessState.MILLISECOND;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MILLISECOND_CHAR,
+ MAX_MILLISECOND_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+
+ case AMPM_CHAR:
+ processState = DateTimeProcessState.AMPM;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, AMPM_CHAR,
+ MAX_AMPM_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+
+ case TIMEZONE_CHAR:
+ processState = DateTimeProcessState.TIMEZONE;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, TIMEZONE_CHAR,
+ MAX_TIMEZONE_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case SKIPPER_CHAR:
+ processState = DateTimeProcessState.SKIPPER;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, SKIPPER_CHAR,
+ MAX_SKIPPER_CHAR);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case ' ':
+ case HYPHEN_CHAR:
+ case COLON_CHAR:
+ case SOLIDUS_CHAR:
+ case PERIOD_CHAR:
+ case COMMA_CHAR:
+ case T_CHAR:
+ // separator
+ separatorChar = format[formatStart + formatPointer];
+ processState = DateTimeProcessState.SEPARATOR;
+ formatPointer++;
+ formatCharCopies++;
+ while (formatPointer < formatLength && format[formatStart + formatPointer] == separatorChar) {
+ formatPointer++;
+ formatCharCopies++;
+ }
+ break;
+
+ default:
+ throw new AsterixTemporalTypeParseException("Unexpected date format string at "
+ + (formatStart + formatPointer) + ": " + format[formatStart + formatPointer]);
+ }
+
+ // check whether the process state is valid for the parse mode
+
+ switch (processState) {
+ case YEAR:
+ case MONTH:
+ case DAY:
+ if (parseMode == DateTimeParseMode.TIME_ONLY) {
+ throw new AsterixTemporalTypeParseException(
+ "Unexpected date format string when parsing a time value");
+ }
+ break;
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ case MILLISECOND:
+ case AMPM:
+ case TIMEZONE:
+ if (parseMode == DateTimeParseMode.DATE_ONLY) {
+ throw new AsterixTemporalTypeParseException(
+ "Unexpected time format string when parsing a date value");
+ }
+ break;
+ default:
+ // do nothing
+ }
+
+ switch (processState) {
+ case INIT:
+ break;
+ case YEAR:
+ if (dataStringPointer < dataLength && data[dataStart + dataStringPointer] == HYPHEN_CHAR) {
+ negativeYear = true;
+ dataStringPointer++;
+ }
+ case DAY:
+ int maxAllowedFormatCharCopies = (processState == DateTimeProcessState.YEAR) ? 4 : 2;
+ int parsedValue = 0;
+
+ int processedFieldsCount = 0;
+ for (int i = 0; i < formatCharCopies; i++) {
+ if (data[dataStart + dataStringPointer] < '0' || data[dataStart + dataStringPointer] > '9') {
+ throw new AsterixTemporalTypeParseException("Unexpected char for year field at "
+ + (dataStart + dataStringPointer) + ": " + data[dataStart + dataStringPointer]);
+ }
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ processedFieldsCount++;
+ }
+ // for more digits
+ while (processedFieldsCount < maxAllowedFormatCharCopies && dataStringPointer < dataLength
+ && data[dataStart + dataStringPointer] >= '0' && data[dataStart + dataStringPointer] <= '9') {
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ processedFieldsCount++;
+ }
+ if (processState == DateTimeProcessState.YEAR) {
+ year = parsedValue;
+ if (negativeYear) {
+ year *= -1;
+ }
+ } else {
+ day = parsedValue;
+ }
+ break;
+ case MONTH:
+ if (formatCharCopies == 3) {
+ // the month is in the text format
+ int monthNameMatch = monthIDSearch(data, dataStart + dataStringPointer, 3);
+ if (monthNameMatch >= 0) {
+ month = monthNameMatch + 1;
+ dataStringPointer += 3;
+ } else {
+ throw new AsterixTemporalTypeParseException("Unrecognizable month string "
+ + (char) data[dataStart + dataStringPointer] + " "
+ + (char) data[dataStart + dataStringPointer + 1] + " "
+ + (char) data[dataStart + dataStringPointer + 2]);
+ }
+ } else {
+ int processedMonthFieldsCount = 0;
+ for (int i = 0; i < formatCharCopies; i++) {
+ if (data[dataStart + dataStringPointer] < '0' || data[dataStart + dataStringPointer] > '9') {
+ throw new AsterixTemporalTypeParseException("Unexpected char for month field at "
+ + (dataStart + dataStringPointer) + ": " + data[dataStart + dataStringPointer]);
+ }
+ month = month * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ if (processedMonthFieldsCount++ > 2) {
+ throw new AsterixTemporalTypeParseException("Unexpected char for month field at "
+ + (dataStart + dataStringPointer) + ": " + data[dataStart + dataStringPointer]);
+ }
+ }
+ // if there are more than 2 digits for the day string
+ while (processedMonthFieldsCount < 2 && dataStringPointer < dataLength
+ && data[dataStart + dataStringPointer] >= '0'
+ && data[dataStart + dataStringPointer] <= '9') {
+ month = month * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ processedMonthFieldsCount++;
+ }
+ }
+ break;
+ case WEEKDAY:
+ int processedWeekdayFieldsCount = 0;
+ while ((data[dataStart + dataStringPointer + processedWeekdayFieldsCount] >= 'a' && data[dataStart
+ + dataStringPointer + processedWeekdayFieldsCount] <= 'z')
+ || (data[dataStart + dataStringPointer + processedWeekdayFieldsCount] >= 'A' && data[dataStart
+ + dataStringPointer + processedWeekdayFieldsCount] <= 'Z')) {
+ processedWeekdayFieldsCount++;
+ }
+ // match the weekday name
+ if (weekdayIDSearch(data, dataStart + dataStringPointer, processedWeekdayFieldsCount) < 0) {
+ throw new AsterixTemporalTypeParseException("Unexpected string for day-of-week: "
+ + (new String(Arrays.copyOfRange(data, dataStart + dataStringPointer, dataStart
+ + dataStringPointer + processedWeekdayFieldsCount))));
+ }
+ dataStringPointer += processedWeekdayFieldsCount;
+ break;
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ case MILLISECOND:
+ int processFieldsCount = 0;
+ int expectedMaxCount = (processState == DateTimeProcessState.MILLISECOND) ? 3 : 2;
+ parsedValue = 0;
+ for (int i = 0; i < formatCharCopies; i++) {
+ if (data[dataStart + dataStringPointer] < '0' || data[dataStart + dataStringPointer] > '9') {
+ throw new AsterixTemporalTypeParseException("Unexpected char for " + processState.name()
+ + " field at " + (dataStart + dataStringPointer) + ": "
+ + data[dataStart + dataStringPointer]);
+ }
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ if (processFieldsCount++ > expectedMaxCount) {
+ throw new AsterixTemporalTypeParseException("Unexpected char for " + processState.name()
+ + " field at " + dataStringPointer + ": " + data[dataStart + dataStringPointer]);
+ }
+ }
+ // if there are more than formatCharCopies digits for the hour string
+ while (processFieldsCount < expectedMaxCount && dataStringPointer < dataLength
+ && data[dataStart + dataStringPointer] >= '0' && data[dataStart + dataStringPointer] <= '9') {
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ processFieldsCount++;
+ }
+ if (processState == DateTimeProcessState.HOUR) {
+ hour = parsedValue;
+ } else if (processState == DateTimeProcessState.MINUTE) {
+ min = parsedValue;
+ } else if (processState == DateTimeProcessState.SECOND) {
+ sec = parsedValue;
+ } else if (processState == DateTimeProcessState.MILLISECOND) {
+ ms = parsedValue;
+ for (int i = processFieldsCount; i < 3; i++) {
+ ms *= 10;
+ }
+ }
+ break;
+ case TIMEZONE:
+ if (data[dataStart + dataStringPointer] == 'Z'
+ && ((dataStringPointer + 1 >= dataLength) || (data[dataStart + dataStringPointer + 1] < 'A'
+ && data[dataStart + dataStringPointer + 1] > 'Z'
+ && data[dataStart + dataStringPointer + 1] < 'a' && data[dataStart
+ + dataStringPointer + 1] > 'z'))) {
+ // UTC as Z
+ timezone = 0;
+ dataStringPointer++;
+ } else if ((data[dataStart + dataStringPointer] == '+' || data[dataStart + dataStringPointer] == '-')
+ || (dataStringPointer + 3 < dataLength && (data[dataStart + dataStringPointer + 3] == '+' || data[dataStart
+ + dataStringPointer + 3] == '-'))) {
+ // UTC+ or GMT+ format
+ if (dataStringPointer + 3 < dataLength
+ && (byteArrayEqualToString(data, dataStart + dataStringPointer, 3, UTC_BYTEARRAY) || byteArrayEqualToString(
+ data, dataStart + dataStringPointer, 3, GMT_BYTEARRAY))) {
+ dataStringPointer += 3;
+ }
+ // parse timezone as +zz:zz or +zzzz
+ boolean negativeTimeZone = false;
+ if (data[dataStart + dataStringPointer] == '-') {
+ negativeTimeZone = true;
+ dataStringPointer++;
+ } else if (data[dataStart + dataStringPointer] == '+') {
+ dataStringPointer++;
+ } else {
+ throw new AsterixTemporalTypeParseException(
+ "Incorrect timezone hour field: expecting sign + or - but got: "
+ + data[dataStart + dataStringPointer]);
+ }
+ // timezone hours
+ for (int i = 0; i < 2; i++) {
+ if (data[dataStart + dataStringPointer + i] >= '0'
+ && data[dataStart + dataStringPointer + i] <= '9') {
+ timezone += (data[dataStart + dataStringPointer + i] - '0') * MS_PER_HOUR;
+ } else {
+ throw new AsterixTemporalTypeParseException(
+ "Unexpected character for timezone hour field at "
+ + (dataStart + dataStringPointer) + ": "
+ + data[dataStart + dataStringPointer]);
+ }
+ }
+ dataStringPointer += 2;
+ // skip the ":" separator
+ if (data[dataStart + dataStringPointer] == ':') {
+ dataStringPointer++;
+ }
+ // timezone minutes
+ for (int i = 0; i < 2; i++) {
+ if (data[dataStart + dataStringPointer + i] >= '0'
+ && data[dataStart + dataStringPointer + i] <= '9') {
+ timezone += (data[dataStart + dataStringPointer + i] - '0') * MS_PER_MINUTE;
+ } else {
+ throw new AsterixTemporalTypeParseException(
+ "Unexpected character for timezone minute field at "
+ + (dataStart + dataStringPointer) + ": "
+ + data[dataStart + dataStringPointer]);
+ }
+ }
+ dataStringPointer += 2;
+ if (!negativeTimeZone) {
+ timezone *= -1;
+ }
+ } else {
+ // do lookup from the zoneinfor database
+ int timezoneEndField = dataStringPointer;
+ while (timezoneEndField < dataLength
+ && ((data[dataStart + timezoneEndField] >= '0' && data[dataStart + timezoneEndField] <= '9')
+ || (data[dataStart + timezoneEndField] >= 'a' && data[dataStart
+ + timezoneEndField] <= 'z')
+ || (data[dataStart + timezoneEndField] >= 'A' && data[dataStart
+ + timezoneEndField] <= 'Z')
+ || data[dataStart + timezoneEndField] == '/' || data[dataStart
+ + timezoneEndField] == '_')) {
+ timezoneEndField++;
+ }
+ int searchIdx = binaryTimezoneIDSearch(data, dataStart + dataStringPointer, timezoneEndField
+ - dataStringPointer);
+ if (searchIdx >= 0) {
+ timezone = TIMEZONE_OFFSETS[searchIdx];
+ } else {
+ throw new AsterixTemporalTypeParseException("Unexpected timezone string: "
+ + new String(Arrays.copyOfRange(data, dataStart + dataStringPointer, dataStart
+ + dataStringPointer)));
+ }
+ dataStringPointer = timezoneEndField;
+ }
+ break;
+ case AMPM:
+ if (dataStringPointer + 1 < dataLength) {
+ if (hour > 12 || hour <= 0) {
+ throw new IllegalStateException("Hour " + hour + " cannot be a time for AM.");
+ }
+ if (byteArrayEqualToString(data, dataStart + dataStringPointer, 2, AM_BYTEARRAY)) {
+ // do nothing
+ } else if (byteArrayEqualToString(data, dataStart + dataStringPointer, 2, PM_BYTEARRAY)) {
+ hour += 12;
+ if (hour == 24) {
+ hour = 0;
+ }
+ } else {
+ throw new AsterixTemporalTypeParseException("Unexpected string for AM/PM marker "
+ + new String(Arrays.copyOfRange(data, dataStart + dataStringPointer, dataStart
+ + dataStringPointer + 2)));
+ }
+ dataStringPointer += 2;
+ } else {
+ throw new AsterixTemporalTypeParseException("Cannot find valid AM/PM marker.");
+ }
+ break;
+ case SKIPPER:
+ // just skip all continuous character and numbers
+ while ((data[dataStart + dataStringPointer] >= 'a' && data[dataStart + dataStringPointer] <= 'z')
+ || (data[dataStart + dataStringPointer] >= 'A' && data[dataStart + dataStringPointer] <= 'Z')
+ || (data[dataStart + dataStringPointer] >= '0' && data[dataStart + dataStringPointer] <= '9')) {
+ dataStringPointer++;
+ }
+ break;
+ case SEPARATOR:
+ if (separatorChar == '\0') {
+ throw new AsterixTemporalTypeParseException("Incorrect separator char in date string as "
+ + data[dataStart + dataStringPointer]);
+ }
+ for (int i = 0; i < formatCharCopies; i++) {
+ if (data[dataStart + dataStringPointer] != separatorChar) {
+ throw new AsterixTemporalTypeParseException("Expecting separator " + separatorChar
+ + " but got " + data[dataStart + dataStringPointer]);
+ }
+ dataStringPointer++;
+ }
+ break;
+ default:
+ throw new AsterixTemporalTypeParseException(
+ "Unexpected time format information when parsing a date value");
+ }
+ }
+
+ if (dataStringPointer < dataLength) {
+ throw new AsterixTemporalTypeParseException(
+ "The given data string is not fully parsed by the given format string");
+ }
+
+ if (formatPointer < formatLength) {
+ throw new AsterixTemporalTypeParseException(
+ "The given format string is not fully used for the given format string");
+ }
+
+ if (parseMode == DateTimeParseMode.TIME_ONLY) {
+ return CAL.getChronon(hour, min, sec, ms, timezone);
+ }
+ return CAL.getChronon(year, month, day, hour, min, sec, ms, timezone);
+ }
+
+ public void printDateTime(long chronon, int timezone, byte[] format, int formatStart, int formatLength,
+ Appendable appender, DateTimeParseMode parseMode) throws HyracksDataException {
+ int year = CAL.getYear(chronon);
+ int month = CAL.getMonthOfYear(chronon, year);
+ int day = CAL.getDayOfMonthYear(chronon, year, month);
+ int hour = CAL.getHourOfDay(chronon);
+ int min = CAL.getMinOfHour(chronon);
+ int sec = CAL.getSecOfMin(chronon);
+ int ms = CAL.getMillisOfSec(chronon);
+
+ int formatCharCopies = 0;
+
+ int formatPointer = 0;
+
+ byte separatorChar = '\0';
+
+ DateTimeProcessState processState = DateTimeProcessState.INIT;
+
+ int pointerMove = 0;
+
+ boolean usePM = false;
+ if (indexOf(format, formatStart, formatLength, 'a') >= 0) {
+ if (hour >= 12) {
+ usePM = true;
+ hour -= 12;
+ }
+ if (hour == 0) {
+ hour = 12;
+ }
+ }
+
+ while (formatPointer < formatLength) {
+
+ formatCharCopies = 0;
+
+ switch (format[formatStart + formatPointer]) {
+ case YEAR_CHAR:
+ processState = DateTimeProcessState.YEAR;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, YEAR_CHAR,
+ MAX_YEAR_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MONTH_CHAR:
+ processState = DateTimeProcessState.MONTH;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MONTH_CHAR,
+ MAX_MONTH_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case DAY_CHAR:
+ processState = DateTimeProcessState.DAY;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, DAY_CHAR,
+ MAX_DAY_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case HOUR_CHAR:
+ processState = DateTimeProcessState.HOUR;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, HOUR_CHAR,
+ MAX_HOUR_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MINUTE_CHAR:
+ processState = DateTimeProcessState.MINUTE;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MINUTE_CHAR,
+ MAX_MINUTE_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case SECOND_CHAR:
+ processState = DateTimeProcessState.SECOND;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, SECOND_CHAR,
+ MAX_SECOND_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case MILLISECOND_CHAR:
+ processState = DateTimeProcessState.MILLISECOND;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MILLISECOND_CHAR,
+ MAX_MILLISECOND_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+
+ case AMPM_CHAR:
+ processState = DateTimeProcessState.AMPM;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, AMPM_CHAR,
+ MAX_AMPM_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+
+ case TIMEZONE_CHAR:
+ processState = DateTimeProcessState.TIMEZONE;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, TIMEZONE_CHAR,
+ MAX_TIMEZONE_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case ' ':
+ case HYPHEN_CHAR:
+ case COLON_CHAR:
+ case SOLIDUS_CHAR:
+ case PERIOD_CHAR:
+ case COMMA_CHAR:
+ case T_CHAR:
+ // separator
+ separatorChar = format[formatStart + formatPointer];
+ processState = DateTimeProcessState.SEPARATOR;
+ formatPointer++;
+ formatCharCopies++;
+ while (formatPointer < formatLength && format[formatStart + formatPointer] == separatorChar) {
+ formatPointer++;
+ formatCharCopies++;
+ }
+ break;
+
+ default:
+ throw new HyracksDataException("Unexpected format string at " + (formatStart + formatPointer)
+ + ": " + format[formatStart + formatPointer]);
+ }
+
+ // check whether the process state is valid for the parse mode
+
+ switch (processState) {
+ case YEAR:
+ case MONTH:
+ case DAY:
+ if (parseMode == DateTimeParseMode.TIME_ONLY) {
+ throw new HyracksDataException("Unexpected date format string when parsing a time value");
+ }
+ break;
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ case MILLISECOND:
+ case AMPM:
+ case TIMEZONE:
+ if (parseMode == DateTimeParseMode.DATE_ONLY) {
+ throw new HyracksDataException("Unexpected time format string when parsing a date value");
+ }
+ break;
+ default:
+ // do nothing
+ }
+
+ try {
+ switch (processState) {
+ case INIT:
+ break;
+ case YEAR:
+ if (year < 0) {
+ appender.append('-');
+ year *= -1;
+ }
+ case MONTH:
+ if (processState == DateTimeProcessState.MONTH && formatCharCopies == 3) {
+ for (byte b : MONTH_NAMES[month - 1]) {
+ appender.append((char) toUpper(b));
+ }
+ break;
+ }
+ case DAY:
+ int val = 0;
+ if (processState == DateTimeProcessState.YEAR) {
+ val = year;
+ } else if (processState == DateTimeProcessState.MONTH) {
+ val = month;
+ } else {
+ val = day;
+ }
+ int valFieldCount = (int) Math.ceil(Math.log10(val));
+ if (val == 1 || val == 0) {
+ valFieldCount = 1;
+ }
+ for (int i = 0; i < formatCharCopies - valFieldCount; i++) {
+ appender.append('0');
+ }
+ appender.append(String.valueOf(val));
+ break;
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ val = 0;
+ if (processState == DateTimeProcessState.HOUR) {
+ val = hour;
+ } else if (processState == DateTimeProcessState.MINUTE) {
+ val = min;
+ } else if (processState == DateTimeProcessState.SECOND) {
+ val = sec;
+ }
+
+ if (val < 10) {
+ for (int i = 0; i < formatCharCopies - 1; i++) {
+ appender.append('0');
+ }
+ }
+ appender.append(String.valueOf(val));
+ break;
+ case MILLISECOND:
+ int msFieldCount = (int) Math.ceil(Math.log10(ms));
+ for (int i = 0; i < 3 - msFieldCount; i++) {
+ appender.append('0');
+ }
+ if (formatCharCopies < 3) {
+
+ if (formatCharCopies == 1) {
+ if (ms % 100 == 0) {
+ // the tailing two zeros can be removed
+ ms = ms / 100;
+ } else if (ms % 10 == 0) {
+ // the tailing one zero can be removed
+ ms = ms / 10;
+ }
+ } else {
+ if (ms % 10 == 0) {
+ // the tailing one zero can be removed
+ ms = ms / 10;
+ }
+ }
+
+ }
+ appender.append(String.valueOf(ms));
+ break;
+ case TIMEZONE:
+ if (timezone == 0) {
+ appender.append('Z');
+ break;
+ }
+ if (timezone < 0) {
+ appender.append('-');
+ timezone *= -1;
+ }
+ int timezoneField = timezone / MS_PER_HOUR;
+ if (timezoneField < 10) {
+ appender.append('0');
+ }
+ appender.append(String.valueOf(timezoneField));
+ timezoneField = timezone % MS_PER_HOUR / MS_PER_MINUTE;
+ if (timezoneField < 10) {
+ appender.append('0');
+ }
+ appender.append(String.valueOf(timezoneField));
+ break;
+ case AMPM:
+ if (usePM) {
+ appender.append("PM");
+ } else {
+ appender.append("AM");
+ }
+ break;
+ case SEPARATOR:
+ if (separatorChar == '\0') {
+ throw new HyracksDataException(
+ "Incorrect separator: separator char is not initialized properly!");
+ }
+ for (int i = 0; i < formatCharCopies; i++) {
+ appender.append((char) separatorChar);
+ }
+ break;
+ default:
+ throw new HyracksDataException("Unexpected time state when printing a date value");
+ }
+ } catch (IOException ex) {
+ throw new HyracksDataException(ex);
+ }
+ }
+ }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/DurationArithmeticOperations.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/DurationArithmeticOperations.java
index e9271e5..d760292 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/DurationArithmeticOperations.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/DurationArithmeticOperations.java
@@ -26,17 +26,27 @@
* <a
* href="http://www.w3.org/TR/xmlschema-2/#adding-durations-to-dateTimes">"XML: adding durations to dateTimes"</a>.
* <p/>
- * The basic algorithm is like this: duration is applied to the time point as two separated fields: year-month field
- * and day-time field. Year-month field is applied firstly by reserving the correct day within the month's range
- * (for example add 1M to 03-31 will return 04-30). Then day-time field is applied.
+ * The basic algorithm is like this: duration is applied to the time point as two separated fields: year-month field and day-time field. Year-month field is applied firstly by reserving the correct day within the month's range (for example add 1M to 03-31 will return 04-30). Then day-time field is applied.
* <p/>
*
* @param pointChronon
+ * The time instance where the duration will be added, represented as the milliseconds since the anchored time (00:00:00 for time type, 1970-01-01T00:00:00Z for datetime and date types).
* @param yearMonthDuration
+ * The year-month-duration to be added
* @param dayTimeDuration
+ * The day-time-duration to be added
* @return
*/
- public static long addDuration(long pointChronon, int yearMonthDuration, long dayTimeDuration) {
+ public static long addDuration(long pointChronon, int yearMonthDuration, long dayTimeDuration, boolean isTimeOnly) {
+
+ if (isTimeOnly) {
+ int rtnChronon = (int) ((pointChronon + dayTimeDuration) % GregorianCalendarSystem.CHRONON_OF_DAY);
+ if (rtnChronon < 0) {
+ rtnChronon += GregorianCalendarSystem.CHRONON_OF_DAY;
+ }
+
+ return rtnChronon;
+ }
int year = calSystem.getYear(pointChronon);
int month = calSystem.getMonthOfYear(pointChronon, year);
@@ -75,13 +85,4 @@
return calSystem.getChronon(year, month, day, hour, min, sec, ms, 0) + dayTimeDuration;
}
- public static int addDuration(int pointChronon, long dayTimeDuration) {
- int rtnChronon = (int) ((pointChronon + dayTimeDuration) % GregorianCalendarSystem.CHRONON_OF_DAY);
- if (rtnChronon < 0) {
- rtnChronon += GregorianCalendarSystem.CHRONON_OF_DAY;
- }
-
- return rtnChronon;
- }
-
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java
index 5d27fb0..3be0858 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/GregorianCalendarSystem.java
@@ -164,15 +164,8 @@
* @return
*/
public boolean validateTimeZone(int timezone) {
- short tzMin = (short) ((timezone % 4) * 15);
- if (tzMin < -60 || tzMin >= 60) {
- return false;
- }
-
- short tzHr = (short) (timezone / 4);
-
- if (tzHr < -12 && tzHr > 14) {
+ if (timezone < -12 * CHRONON_OF_DAY || timezone > 14 * CHRONON_OF_DAY) {
return false;
}
@@ -211,8 +204,8 @@
*/
public long getChronon(int year, int month, int day, int hour, int min, int sec, int millis, int timezone) {
// Added milliseconds for all fields but month and day
- long chrononTime = chrononizeBeginningOfYear(year) + (hour - timezone / 4) * CHRONON_OF_HOUR
- + (min - (timezone % 4) * 15) * CHRONON_OF_MINUTE + sec * CHRONON_OF_SECOND + millis;
+ long chrononTime = chrononizeBeginningOfYear(year) + hour * CHRONON_OF_HOUR + min * CHRONON_OF_MINUTE + sec
+ * CHRONON_OF_SECOND + millis + timezone;
// Added milliseconds for days of the month.
chrononTime += (day - 1 + DAYS_SINCE_MONTH_BEGIN_ORDI[month - 1]) * CHRONON_OF_DAY;
@@ -237,13 +230,13 @@
*/
public int getChronon(int hour, int min, int sec, int millis, int timezone) {
// Added milliseconds for all fields but month and day
- long chrononTime = (hour - timezone / 4) * CHRONON_OF_HOUR + (min - (timezone % 4) * 15) * CHRONON_OF_MINUTE
- + sec * CHRONON_OF_SECOND + millis;
- return (int)chrononTime;
+ long chrononTime = hour * CHRONON_OF_HOUR + min * CHRONON_OF_MINUTE + sec * CHRONON_OF_SECOND + millis
+ + timezone;
+ return (int) chrononTime;
}
public long adjustChrononByTimezone(long chronon, int timezone) {
- return chronon + timezone / 4 * CHRONON_OF_HOUR + (timezone % 4) * 15 * CHRONON_OF_MINUTE;
+ return chronon - timezone;
}
public static int getChrononInDays(long chronon) {
@@ -330,12 +323,12 @@
if (timezone == 0) {
sbder.append("Z");
} else {
- short tzMin = (short) ((timezone % 4) * 15);
+ int tzMin = (int) (timezone % CHRONON_OF_HOUR / CHRONON_OF_MINUTE);
if (tzMin < 0) {
tzMin = (short) (-1 * tzMin);
}
- short tzHr = (short) (timezone / 4);
- sbder.append((tzHr >= 0 ? "+" : "-")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
+ int tzHr = (int) (timezone / CHRONON_OF_HOUR);
+ sbder.append((tzHr >= 0 ? "-" : "+")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
.append(":").append(String.format("%02d", tzMin));
}
}
@@ -391,12 +384,12 @@
if (timezone == 0) {
sbder.append("Z");
} else {
- short tzMin = (short) ((timezone % 4) * 15);
+ int tzMin = (int) (timezone % CHRONON_OF_HOUR / CHRONON_OF_MINUTE);
if (tzMin < 0) {
tzMin = (short) (-1 * tzMin);
}
- short tzHr = (short) (timezone / 4);
- sbder.append((tzHr >= 0 ? "+" : "-")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
+ int tzHr = (int) (timezone / CHRONON_OF_HOUR);
+ sbder.append((tzHr >= 0 ? "-" : "+")).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
.append(String.format("%02d", tzMin));
}
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
index 42acae6..540530e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -55,10 +55,11 @@
import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedFieldAccessByNameResultType;
import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedGetItemResultType;
import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedLocalAvgTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedMinMaxAggTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedNumericAddSubMulDivTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedNumericRoundHalfToEven2TypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedNumericUnaryFunctionTypeComputer;
-import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedSumTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedNumericAggTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedSwitchCaseComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedUnaryMinusTypeComputer;
import edu.uci.ics.asterix.om.typecomputer.impl.NotNullTypeComputer;
@@ -172,8 +173,6 @@
FunctionConstants.ASTERIX_NS, "closed-record-constructor", FunctionIdentifier.VARARGS);
public final static FunctionIdentifier OPEN_RECORD_CONSTRUCTOR = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "open-record-constructor", FunctionIdentifier.VARARGS);
- public final static FunctionIdentifier RECORD_TYPE_CONSTRUCTOR = new FunctionIdentifier(
- FunctionConstants.ASTERIX_NS, "record-type-constructor", FunctionIdentifier.VARARGS);
public final static FunctionIdentifier FIELD_ACCESS_BY_INDEX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"field-access-by-index", 2);
public final static FunctionIdentifier FIELD_ACCESS_BY_NAME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
@@ -522,32 +521,22 @@
FunctionConstants.ASTERIX_NS, "get-interval-start", 1);
public static final FunctionIdentifier ACCESSOR_TEMPORAL_INTERVAL_END = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "get-interval-end", 1);
+ public static final FunctionIdentifier INTERVAL_BIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "interval-bin", 3);
// Temporal functions
public static final FunctionIdentifier DATE_FROM_UNIX_TIME_IN_DAYS = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "date-from-unix-time-in-days", 1);
public static final FunctionIdentifier DATE_FROM_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"get-date-from-datetime", 1);
- public final static FunctionIdentifier ADD_DATE_DURATION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- "add-date-duration", 2);
- public final static FunctionIdentifier SUBTRACT_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- "subtract-date", 2);
public final static FunctionIdentifier TIME_FROM_UNIX_TIME_IN_MS = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "time-from-unix-time-in-ms", 1);
public final static FunctionIdentifier TIME_FROM_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"get-time-from-datetime", 1);
- public final static FunctionIdentifier SUBTRACT_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- "subtract-time", 2);
- public final static FunctionIdentifier ADD_TIME_DURATION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- "add-time-duration", 2);
public final static FunctionIdentifier DATETIME_FROM_UNIX_TIME_IN_MS = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "datetime-from-unix-time-in-ms", 1);
public final static FunctionIdentifier DATETIME_FROM_DATE_TIME = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "datetime-from-date-time", 2);
- public final static FunctionIdentifier SUBTRACT_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- "subtract-datetime", 2);
- public final static FunctionIdentifier ADD_DATETIME_DURATION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
- "add-datetime-duration", 2);
public final static FunctionIdentifier CALENDAR_DURATION_FROM_DATETIME = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "calendar-duration-from-datetime", 2);
public final static FunctionIdentifier CALENDAR_DURATION_FROM_DATE = new FunctionIdentifier(
@@ -556,6 +545,20 @@
FunctionConstants.ASTERIX_NS, "adjust-time-for-timezone", 2);
public final static FunctionIdentifier ADJUST_DATETIME_FOR_TIMEZONE = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "adjust-datetime-for-timezone", 2);
+ public final static FunctionIdentifier DAY_OF_WEEK = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "day-of-week");
+ public final static FunctionIdentifier PARSE_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "parse-date", 2);
+ public final static FunctionIdentifier PARSE_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "parse-time", 2);
+ public final static FunctionIdentifier PARSE_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "parse-datetime", 2);
+ public final static FunctionIdentifier PRINT_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "print-date", 2);
+ public final static FunctionIdentifier PRINT_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "print-time", 2);
+ public final static FunctionIdentifier PRINT_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "print-datetime", 2);
public final static FunctionIdentifier GET_POINT_X_COORDINATE_ACCESSOR = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "get-x", 1);
@@ -616,20 +619,20 @@
// and then, Asterix builtin functions
addPrivateFunction(NOT_NULL, NotNullTypeComputer.INSTANCE);
addPrivateFunction(ANY_COLLECTION_MEMBER, NonTaggedCollectionMemberResultType.INSTANCE);
- addPrivateFunction(AVG, OptionalADoubleTypeComputer.INSTANCE);
+ addFunction(AVG, OptionalADoubleTypeComputer.INSTANCE);
addFunction(BOOLEAN_CONSTRUCTOR, UnaryBooleanOrNullFunctionTypeComputer.INSTANCE);
- addFunction(CARET, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+ addPrivateFunction(CARET, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
addFunction(CIRCLE_CONSTRUCTOR, OptionalACircleTypeComputer.INSTANCE);
addPrivateFunction(CLOSED_RECORD_CONSTRUCTOR, ClosedRecordConstructorResultType.INSTANCE);
addPrivateFunction(CONCAT_NON_NULL, ConcatNonNullTypeComputer.INSTANCE);
addFunction(CONTAINS, ABooleanTypeComputer.INSTANCE);
- addPrivateFunction(COUNT, AInt64TypeComputer.INSTANCE);
- addFunction(COUNTHASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
+ addFunction(COUNT, AInt64TypeComputer.INSTANCE);
+ addPrivateFunction(COUNTHASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
addPrivateFunction(COUNTHASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
addFunction(CREATE_CIRCLE, ACircleTypeComputer.INSTANCE);
addFunction(CREATE_LINE, ALineTypeComputer.INSTANCE);
- addFunction(CREATE_MBR, ADoubleTypeComputer.INSTANCE);
+ addPrivateFunction(CREATE_MBR, ADoubleTypeComputer.INSTANCE);
addFunction(CREATE_POINT, APointTypeComputer.INSTANCE);
addFunction(CREATE_POLYGON, APolygonTypeComputer.INSTANCE);
addFunction(CREATE_RECTANGLE, ARectangleTypeComputer.INSTANCE);
@@ -664,7 +667,7 @@
addPrivateFunction(GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
addPrivateFunction(GRAM_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE);
addFunction(GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
- addFunction(HASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
+ addPrivateFunction(HASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
addPrivateFunction(HASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
addPrivateFunction(INDEX_SEARCH, new IResultTypeComputer() {
@@ -685,18 +688,18 @@
addPrivateFunction(LOCAL_AVG, NonTaggedLocalAvgTypeComputer.INSTANCE);
addPrivateFunction(MAKE_FIELD_INDEX_HANDLE, null); // TODO
addPrivateFunction(MAKE_FIELD_NAME_HANDLE, null); // TODO
- addPrivateFunction(MAX, NonTaggedSumTypeComputer.INSTANCE);
- addPrivateFunction(LOCAL_MAX, NonTaggedSumTypeComputer.INSTANCE);
- addPrivateFunction(MIN, NonTaggedSumTypeComputer.INSTANCE);
- addPrivateFunction(LOCAL_MIN, NonTaggedSumTypeComputer.INSTANCE);
+ addFunction(MAX, NonTaggedMinMaxAggTypeComputer.INSTANCE);
+ addPrivateFunction(LOCAL_MAX, NonTaggedMinMaxAggTypeComputer.INSTANCE);
+ addFunction(MIN, NonTaggedMinMaxAggTypeComputer.INSTANCE);
+ addPrivateFunction(LOCAL_MIN, NonTaggedMinMaxAggTypeComputer.INSTANCE);
addPrivateFunction(NON_EMPTY_STREAM, ABooleanTypeComputer.INSTANCE);
addFunction(NULL_CONSTRUCTOR, ANullTypeComputer.INSTANCE);
- addFunction(NUMERIC_UNARY_MINUS, NonTaggedUnaryMinusTypeComputer.INSTANCE);
- addFunction(NUMERIC_SUBTRACT, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
- addFunction(NUMERIC_MULTIPLY, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
- addFunction(NUMERIC_DIVIDE, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
- addFunction(NUMERIC_MOD, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
- addFunction(NUMERIC_IDIV, AInt32TypeComputer.INSTANCE);
+ addPrivateFunction(NUMERIC_UNARY_MINUS, NonTaggedUnaryMinusTypeComputer.INSTANCE);
+ addPrivateFunction(NUMERIC_SUBTRACT, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+ addPrivateFunction(NUMERIC_MULTIPLY, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+ addPrivateFunction(NUMERIC_DIVIDE, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+ addPrivateFunction(NUMERIC_MOD, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
+ addPrivateFunction(NUMERIC_IDIV, AInt32TypeComputer.INSTANCE);
addFunction(NUMERIC_ABS, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
addFunction(NUMERIC_CEILING, NonTaggedNumericUnaryFunctionTypeComputer.INSTANCE);
@@ -730,7 +733,6 @@
addPrivateFunction(PREFIX_LEN_JACCARD, AInt32TypeComputer.INSTANCE);
addFunction(RANGE, AInt32TypeComputer.INSTANCE);
addFunction(RECTANGLE_CONSTRUCTOR, OptionalARectangleTypeComputer.INSTANCE);
- // add(RECORD_TYPE_CONSTRUCTOR, null);
addFunction(SCALAR_AVG, ScalarVersionOfAggregateResultType.INSTANCE);
addFunction(SCALAR_COUNT, AInt64TypeComputer.INSTANCE);
@@ -744,8 +746,8 @@
addPrivateFunction(SERIAL_COUNT, AInt64TypeComputer.INSTANCE);
addPrivateFunction(SERIAL_GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
addPrivateFunction(SERIAL_LOCAL_AVG, NonTaggedLocalAvgTypeComputer.INSTANCE);
- addPrivateFunction(SERIAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
- addPrivateFunction(SERIAL_LOCAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
+ addPrivateFunction(SERIAL_SUM, NonTaggedNumericAggTypeComputer.INSTANCE);
+ addPrivateFunction(SERIAL_LOCAL_SUM, NonTaggedNumericAggTypeComputer.INSTANCE);
addFunction(SIMILARITY_JACCARD, AFloatTypeComputer.INSTANCE);
addFunction(SIMILARITY_JACCARD_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
addPrivateFunction(SIMILARITY_JACCARD_SORTED, AFloatTypeComputer.INSTANCE);
@@ -801,8 +803,8 @@
}
});
addFunction(SUBSTRING, SubstringTypeComputer.INSTANCE);
- addPrivateFunction(SUM, NonTaggedSumTypeComputer.INSTANCE);
- addPrivateFunction(LOCAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
+ addFunction(SUM, NonTaggedNumericAggTypeComputer.INSTANCE);
+ addPrivateFunction(LOCAL_SUM, NonTaggedNumericAggTypeComputer.INSTANCE);
addFunction(SWITCH_CASE, NonTaggedSwitchCaseComputer.INSTANCE);
addPrivateFunction(REG_EXP, ABooleanTypeComputer.INSTANCE);
addFunction(INJECT_FAILURE, InjectFailureTypeComputer.INSTANCE);
@@ -811,7 +813,7 @@
addFunction(TID, AInt32TypeComputer.INSTANCE);
addFunction(TIME_CONSTRUCTOR, OptionalATimeTypeComputer.INSTANCE);
- addFunction(TYPE_OF, null); // TODO
+ addPrivateFunction(TYPE_OF, null);
addPrivateFunction(UNORDERED_LIST_CONSTRUCTOR, UnorderedListConstructorResultType.INSTANCE);
addFunction(WORD_TOKENS, new IResultTypeComputer() {
@Override
@@ -835,16 +837,10 @@
// temporal functions
addFunction(DATE_FROM_UNIX_TIME_IN_DAYS, OptionalADateTypeComputer.INSTANCE);
addFunction(DATE_FROM_DATETIME, OptionalADateTypeComputer.INSTANCE);
- addFunction(ADD_DATE_DURATION, OptionalADateTypeComputer.INSTANCE);
- addFunction(SUBTRACT_DATE, OptionalADurationTypeComputer.INSTANCE);
addFunction(TIME_FROM_UNIX_TIME_IN_MS, OptionalATimeTypeComputer.INSTANCE);
addFunction(TIME_FROM_DATETIME, OptionalATimeTypeComputer.INSTANCE);
- addFunction(SUBTRACT_TIME, OptionalADurationTypeComputer.INSTANCE);
- addFunction(ADD_TIME_DURATION, OptionalATimeTypeComputer.INSTANCE);
addFunction(DATETIME_FROM_DATE_TIME, OptionalADateTimeTypeComputer.INSTANCE);
addFunction(DATETIME_FROM_UNIX_TIME_IN_MS, OptionalADateTimeTypeComputer.INSTANCE);
- addFunction(SUBTRACT_DATETIME, OptionalADurationTypeComputer.INSTANCE);
- addFunction(ADD_DATETIME_DURATION, OptionalADateTimeTypeComputer.INSTANCE);
addFunction(CALENDAR_DURATION_FROM_DATETIME, OptionalADurationTypeComputer.INSTANCE);
addFunction(CALENDAR_DURATION_FROM_DATE, OptionalADurationTypeComputer.INSTANCE);
addFunction(ADJUST_DATETIME_FOR_TIMEZONE, OptionalAStringTypeComputer.INSTANCE);
@@ -876,6 +872,14 @@
addFunction(MILLISECONDS_FROM_DAY_TIME_DURATION, OptionalAInt64TypeComputer.INSTANCE);
addFunction(GET_DAY_TIME_DURATION, OptionalADayTimeDurationTypeComputer.INSTANCE);
addFunction(GET_YEAR_MONTH_DURATION, OptionalAYearMonthDurationTypeComputer.INSTANCE);
+ addFunction(INTERVAL_BIN, OptionalAIntervalTypeComputer.INSTANCE);
+ addFunction(DAY_OF_WEEK, OptionalAInt32TypeComputer.INSTANCE);
+ addFunction(PARSE_DATE, OptionalADateTypeComputer.INSTANCE);
+ addFunction(PARSE_TIME, OptionalATimeTypeComputer.INSTANCE);
+ addFunction(PARSE_DATETIME, OptionalADateTimeTypeComputer.INSTANCE);
+ addFunction(PRINT_DATE, OptionalAStringTypeComputer.INSTANCE);
+ addFunction(PRINT_TIME, OptionalAStringTypeComputer.INSTANCE);
+ addFunction(PRINT_DATETIME, OptionalAStringTypeComputer.INSTANCE);
// interval constructors
addFunction(INTERVAL_CONSTRUCTOR_DATE, OptionalAIntervalTypeComputer.INSTANCE);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/cast/ARecordCaster.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/cast/ARecordCaster.java
index 34638c8..507b845 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/cast/ARecordCaster.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/cast/ARecordCaster.java
@@ -278,8 +278,7 @@
// recursively casting, the result of casting can always be thought
// as flat
if (optionalFields[i]) {
- if (fieldTypeTags.size() <= i || fieldTypeTags.get(i) == null
- || fieldTypeTags.get(i).equals(nullTypeTag)) {
+ if (pos == -1 || fieldTypeTags.get(pos) == null || fieldTypeTags.get(pos).equals(nullTypeTag)) {
//the field is optional in the input record
nestedVisitorArg.second = ((AUnionType) fType).getUnionList().get(0);
} else {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedSumTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedMinMaxAggTypeComputer.java
similarity index 88%
copy from asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedSumTypeComputer.java
copy to asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedMinMaxAggTypeComputer.java
index 2437da4..a6c39b6 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedSumTypeComputer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedMinMaxAggTypeComputer.java
@@ -30,13 +30,12 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-public class NonTaggedSumTypeComputer implements IResultTypeComputer {
+public class NonTaggedMinMaxAggTypeComputer implements IResultTypeComputer {
+ private static final String errMsg = "Aggregator is not implemented for ";
- private static final String errMsg = "Sum aggregator is not implemented for ";
+ public static final NonTaggedMinMaxAggTypeComputer INSTANCE = new NonTaggedMinMaxAggTypeComputer();
- public static final NonTaggedSumTypeComputer INSTANCE = new NonTaggedSumTypeComputer();
-
- private NonTaggedSumTypeComputer() {
+ private NonTaggedMinMaxAggTypeComputer() {
}
@Override
@@ -79,6 +78,9 @@
case INT8:
unionList.add(BuiltinType.AINT8);
break;
+ case STRING:
+ unionList.add(BuiltinType.ASTRING);
+ break;
case ANY:
return BuiltinType.ANY;
default: {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericAddSubMulDivTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericAddSubMulDivTypeComputer.java
index 6c46f3a..154ce65 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericAddSubMulDivTypeComputer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericAddSubMulDivTypeComputer.java
@@ -231,6 +231,111 @@
}
}
}
+ case DATE: {
+ switch (tag2) {
+ case DATE:
+ unionList.add(BuiltinType.ADURATION);
+ break;
+ case YEARMONTHDURATION:
+ case DAYTIMEDURATION:
+ case DURATION:
+ unionList.add(BuiltinType.ADATE);
+ break;
+ default: {
+ throw new NotImplementedException(errMsg + tag2);
+ }
+ }
+ break;
+ }
+ case TIME: {
+ switch (tag2) {
+ case TIME:
+ unionList.add(BuiltinType.ADURATION);
+ break;
+ case YEARMONTHDURATION:
+ case DAYTIMEDURATION:
+ case DURATION:
+ unionList.add(BuiltinType.ATIME);
+ break;
+ default: {
+ throw new NotImplementedException(errMsg + tag2);
+ }
+ }
+ break;
+ }
+ case DATETIME: {
+ switch (tag2) {
+ case DATETIME:
+ unionList.add(BuiltinType.ADURATION);
+ break;
+ case YEARMONTHDURATION:
+ case DAYTIMEDURATION:
+ case DURATION:
+ unionList.add(BuiltinType.ADATETIME);
+ break;
+ default: {
+ throw new NotImplementedException(errMsg + tag2);
+ }
+ }
+ break;
+ }
+ case DURATION: {
+ switch(tag2){
+ case DATE:
+ unionList.add(BuiltinType.ADATE);
+ break;
+ case TIME:
+ unionList.add(BuiltinType.ATIME);
+ break;
+ case DATETIME:
+ unionList.add(BuiltinType.ADATETIME);
+ break;
+ default: {
+ throw new NotImplementedException(errMsg + tag2);
+ }
+ }
+ break;
+ }
+ case YEARMONTHDURATION: {
+ switch(tag2){
+ case DATE:
+ unionList.add(BuiltinType.ADATE);
+ break;
+ case TIME:
+ unionList.add(BuiltinType.ATIME);
+ break;
+ case DATETIME:
+ unionList.add(BuiltinType.ADATETIME);
+ break;
+ case YEARMONTHDURATION:
+ unionList.add(BuiltinType.AYEARMONTHDURATION);
+ break;
+ default: {
+ throw new NotImplementedException(errMsg + tag2);
+ }
+ }
+ break;
+ }
+ case DAYTIMEDURATION: {
+ switch(tag2){
+ case DATE:
+ unionList.add(BuiltinType.ADATE);
+ break;
+ case TIME:
+ unionList.add(BuiltinType.ATIME);
+ break;
+ case DATETIME:
+ unionList.add(BuiltinType.ADATETIME);
+ break;
+ case DAYTIMEDURATION:
+ unionList.add(BuiltinType.ADAYTIMEDURATION);
+ break;
+ default: {
+ throw new NotImplementedException(errMsg + tag2);
+ }
+ }
+ break;
+ }
default: {
throw new NotImplementedException(errMsg + tag1);
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedSumTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericAggTypeComputer.java
similarity index 91%
rename from asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedSumTypeComputer.java
rename to asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericAggTypeComputer.java
index 2437da4..7582785 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedSumTypeComputer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedNumericAggTypeComputer.java
@@ -30,13 +30,13 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-public class NonTaggedSumTypeComputer implements IResultTypeComputer {
+public class NonTaggedNumericAggTypeComputer implements IResultTypeComputer {
- private static final String errMsg = "Sum aggregator is not implemented for ";
+ private static final String errMsg = "Aggregator is not implemented for ";
- public static final NonTaggedSumTypeComputer INSTANCE = new NonTaggedSumTypeComputer();
+ public static final NonTaggedNumericAggTypeComputer INSTANCE = new NonTaggedNumericAggTypeComputer();
- private NonTaggedSumTypeComputer() {
+ private NonTaggedNumericAggTypeComputer() {
}
@Override
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
index 1791a80..b3f1332 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
@@ -27,100 +27,92 @@
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
/*
* Acts as an holder class for IndexRegistryProvider, AsterixStorageManager
* instances that are accessed from the NCs. In addition an instance of ICCApplicationContext
* is stored for access by the CC.
*/
-public class AsterixAppContextInfo implements IAsterixApplicationContextInfo,
- IAsterixPropertiesProvider {
+public class AsterixAppContextInfo implements IAsterixApplicationContextInfo, IAsterixPropertiesProvider {
- private static AsterixAppContextInfo INSTANCE;
+ private static AsterixAppContextInfo INSTANCE;
- private final ICCApplicationContext appCtx;
+ private final ICCApplicationContext appCtx;
- private AsterixCompilerProperties compilerProperties;
- private AsterixExternalProperties externalProperties;
- private AsterixMetadataProperties metadataProperties;
- private AsterixStorageProperties storageProperties;
- private AsterixTransactionProperties txnProperties;
- private IHyracksClientConnection hcc;
+ private AsterixCompilerProperties compilerProperties;
+ private AsterixExternalProperties externalProperties;
+ private AsterixMetadataProperties metadataProperties;
+ private AsterixStorageProperties storageProperties;
+ private AsterixTransactionProperties txnProperties;
+ private IHyracksClientConnection hcc;
- public static void initialize(ICCApplicationContext ccAppCtx,
- IHyracksClientConnection hcc) throws AsterixException {
- if (INSTANCE == null) {
- INSTANCE = new AsterixAppContextInfo(ccAppCtx);
- }
- AsterixPropertiesAccessor propertiesAccessor = new AsterixPropertiesAccessor();
- INSTANCE.compilerProperties = new AsterixCompilerProperties(
- propertiesAccessor);
- INSTANCE.externalProperties = new AsterixExternalProperties(
- propertiesAccessor);
- INSTANCE.metadataProperties = new AsterixMetadataProperties(
- propertiesAccessor);
- INSTANCE.storageProperties = new AsterixStorageProperties(
- propertiesAccessor);
- INSTANCE.txnProperties = new AsterixTransactionProperties(
- propertiesAccessor);
- INSTANCE.hcc = hcc;
- Logger.getLogger("edu.uci.ics").setLevel(
- INSTANCE.externalProperties.getLogLevel());
- }
+ public static void initialize(ICCApplicationContext ccAppCtx, IHyracksClientConnection hcc) throws AsterixException {
+ if (INSTANCE == null) {
+ INSTANCE = new AsterixAppContextInfo(ccAppCtx);
+ }
+ AsterixPropertiesAccessor propertiesAccessor = new AsterixPropertiesAccessor();
+ INSTANCE.compilerProperties = new AsterixCompilerProperties(propertiesAccessor);
+ INSTANCE.externalProperties = new AsterixExternalProperties(propertiesAccessor);
+ INSTANCE.metadataProperties = new AsterixMetadataProperties(propertiesAccessor);
+ INSTANCE.storageProperties = new AsterixStorageProperties(propertiesAccessor);
+ INSTANCE.txnProperties = new AsterixTransactionProperties(propertiesAccessor);
+ INSTANCE.hcc = hcc;
+ Logger.getLogger("edu.uci.ics").setLevel(INSTANCE.externalProperties.getLogLevel());
+ }
- private AsterixAppContextInfo(ICCApplicationContext ccAppCtx) {
- this.appCtx = ccAppCtx;
- }
+ private AsterixAppContextInfo(ICCApplicationContext ccAppCtx) {
+ this.appCtx = ccAppCtx;
+ }
- public static AsterixAppContextInfo getInstance() {
- return INSTANCE;
- }
+ public static AsterixAppContextInfo getInstance() {
+ return INSTANCE;
+ }
- @Override
- public ICCApplicationContext getCCApplicationContext() {
- return appCtx;
- }
+ @Override
+ public ICCApplicationContext getCCApplicationContext() {
+ return appCtx;
+ }
- @Override
- public AsterixStorageProperties getStorageProperties() {
- return storageProperties;
- }
+ @Override
+ public AsterixStorageProperties getStorageProperties() {
+ return storageProperties;
+ }
- @Override
- public AsterixTransactionProperties getTransactionProperties() {
- return txnProperties;
- }
+ @Override
+ public AsterixTransactionProperties getTransactionProperties() {
+ return txnProperties;
+ }
- @Override
- public AsterixCompilerProperties getCompilerProperties() {
- return compilerProperties;
- }
+ @Override
+ public AsterixCompilerProperties getCompilerProperties() {
+ return compilerProperties;
+ }
- @Override
- public AsterixMetadataProperties getMetadataProperties() {
- return metadataProperties;
- }
+ @Override
+ public AsterixMetadataProperties getMetadataProperties() {
+ return metadataProperties;
+ }
- @Override
- public AsterixExternalProperties getExternalProperties() {
- return externalProperties;
- }
+ @Override
+ public AsterixExternalProperties getExternalProperties() {
+ return externalProperties;
+ }
- public IHyracksClientConnection getHcc() {
- return hcc;
- }
+ public IHyracksClientConnection getHcc() {
+ return hcc;
+ }
- @Override
- public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
- return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
- }
+ @Override
+ public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
+ return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+ }
- @Override
- public IStorageManagerInterface getStorageManagerInterface() {
- return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
- }
+ @Override
+ public IStorageManagerInterface getStorageManagerInterface() {
+ return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+ }
+
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
index a4f0da2..d6fae85 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
@@ -71,7 +71,7 @@
// Probe phase: Probe items from second list, and compute intersection size.
int intersectionSize = 0;
int probeListCount = 0;
- int minUnionSize = probeListSize;
+ int minUnionSize = buildListSize;
while (probeIter.hasNext()) {
probeListCount++;
byte[] buf = probeIter.getData();
@@ -97,7 +97,7 @@
// Could not find element in other set. Increase min union size by 1.
minUnionSize++;
// Check whether jaccThresh can still be satisfied if there was a mismatch.
- int maxIntersectionSize = intersectionSize + (probeListSize - probeListCount);
+ int maxIntersectionSize = Math.min(buildListSize, intersectionSize + (probeListSize - probeListCount));
int lowerBound = (int) Math.floor(jaccThresh * minUnionSize);
if (maxIntersectionSize < lowerBound) {
// Cannot satisfy jaccThresh.
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
index 03c4140..5070210 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
@@ -83,6 +83,13 @@
int stringLength = (serString[1] & 0xff << 8) + (serString[2] & 0xff << 0);
+ // the string to be parsed should be at least 8 characters: YYYYMMDD
+ if (stringLength < 8) {
+ throw new AlgebricksException(errorMessage
+ + ": the string length should be at least 8 (YYYYMMDD) but it is "
+ + stringLength);
+ }
+
int startOffset = 3;
while (serString[startOffset] == ' ') {
startOffset++;
@@ -111,8 +118,6 @@
}
} catch (IOException e1) {
throw new AlgebricksException(errorMessage);
- } catch (Exception e2) {
- throw new AlgebricksException(e2);
}
}
};
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
index 986b158..880de44 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
@@ -82,7 +82,12 @@
if (serString[0] == SER_STRING_TYPE_TAG) {
int stringLength = (serString[1] & 0xff << 8) + (serString[2] & 0xff << 0);
-
+ // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
+ if (stringLength < 14) {
+ throw new AlgebricksException(errorMessage
+ + ": the string length should be at least 14 (YYYYMMDDhhmmss) but it is "
+ + stringLength);
+ }
// +1 if it is negative (-)
short timeOffset = (short) ((serString[3] == '-') ? 1 : 0);
@@ -109,8 +114,6 @@
}
} catch (IOException e1) {
throw new AlgebricksException(errorMessage);
- } catch (Exception e2) {
- throw new AlgebricksException(e2);
}
}
};
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
index eb33cc4..9b17c90 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
@@ -122,7 +122,7 @@
if (argOut1.getByteArray()[0] == SER_DURATION_TYPE_TAG) {
intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
ADurationSerializerDeserializer.getYearMonth(argOut1.getByteArray(), 1),
- ADurationSerializerDeserializer.getDayTime(argOut1.getByteArray(), 1));
+ ADurationSerializerDeserializer.getDayTime(argOut1.getByteArray(), 1), false);
} else if (argOut1.getByteArray()[0] == SER_STRING_TYPE_TAG) {
// duration
int stringLength = (argOut1.getByteArray()[1] & 0xff << 8)
@@ -131,7 +131,7 @@
ADurationParserFactory.parseDuration(argOut1.getByteArray(), 3, stringLength,
aDuration, ADurationParseOption.All);
intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
- aDuration.getMonths(), aDuration.getMilliseconds());
+ aDuration.getMonths(), aDuration.getMilliseconds(), false);
} else {
throw new AlgebricksException(FID.getName()
+ ": expects NULL/STRING/DURATION for the second argument, but got "
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
index f7b4cdd..ce4681a 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
@@ -134,7 +134,7 @@
if (argOut1.getByteArray()[0] == SER_DURATION_TYPE_TAG) {
intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
ADurationSerializerDeserializer.getYearMonth(argOut1.getByteArray(), 1),
- ADurationSerializerDeserializer.getDayTime(argOut1.getByteArray(), 1));
+ ADurationSerializerDeserializer.getDayTime(argOut1.getByteArray(), 1), false);
} else if (argOut1.getByteArray()[0] == SER_STRING_TYPE_TAG) {
// duration
@@ -145,7 +145,7 @@
aDuration, ADurationParseOption.All);
intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
- aDuration.getMonths(), aDuration.getMilliseconds());
+ aDuration.getMonths(), aDuration.getMilliseconds(), false);
} else {
throw new AlgebricksException(FID.getName()
+ ": expects NULL/STRING/DURATION for the second argument but got "
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
index 3f8e5a8..d4168e1 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
@@ -130,7 +130,7 @@
}
intervalEnd = DurationArithmeticOperations.addDuration(intervalStart, 0,
- ADurationSerializerDeserializer.getDayTime(argOut1.getByteArray(), 1));
+ ADurationSerializerDeserializer.getDayTime(argOut1.getByteArray(), 1), false);
} else if (argOut1.getByteArray()[0] == SER_STRING_TYPE_TAG) {
// duration
@@ -147,7 +147,7 @@
}
intervalEnd = DurationArithmeticOperations.addDuration(intervalStart, 0,
- aDuration.getMilliseconds());
+ aDuration.getMilliseconds(), false);
} else {
throw new AlgebricksException("Wrong format for interval constructor from dates.");
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
index 1b3e7f6..d344d1c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
@@ -83,6 +83,13 @@
int stringLength = (serString[1] & 0xff << 8) + (serString[2] & 0xff << 0);
+ // the string to be parsed should be at least 6 characters: hhmmss
+ if (stringLength < 6) {
+ throw new AlgebricksException(errorMessage
+ + ": the string length should be at least 6 (hhmmss) but it is "
+ + stringLength);
+ }
+
int chrononTimeInMs = ATimeParserFactory.parseTimePart(serString, 3, stringLength);
if (chrononTimeInMs < 0) {
@@ -99,8 +106,6 @@
}
} catch (IOException e1) {
throw new AlgebricksException(errorMessage);
- } catch (Exception e2) {
- throw new AlgebricksException(e2);
}
}
};
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
index 379b934..84b9a32 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
@@ -16,20 +16,31 @@
import java.io.DataOutput;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADayTimeDurationSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableDate;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
import edu.uci.ics.asterix.om.base.AMutableDouble;
+import edu.uci.ics.asterix.om.base.AMutableDuration;
import edu.uci.ics.asterix.om.base.AMutableFloat;
import edu.uci.ics.asterix.om.base.AMutableInt16;
import edu.uci.ics.asterix.om.base.AMutableInt32;
import edu.uci.ics.asterix.om.base.AMutableInt64;
import edu.uci.ics.asterix.om.base.AMutableInt8;
+import edu.uci.ics.asterix.om.base.AMutableTime;
import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.EnumDeserializer;
@@ -51,6 +62,29 @@
abstract protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException;
+ /**
+ * abstract method for arithmetic operation between a time instance (date/time/datetime)
+ * and a duration (duration/year-month-duration/day-time-duration)
+ *
+ * @param chronon
+ * @param yearMonth
+ * @param dayTime
+ * @return
+ * @throws HyracksDataException
+ */
+ abstract protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+ throws HyracksDataException;
+
+ /**
+ * abstract method for arithmetic operation between two time instances (date/time/datetime)
+ *
+ * @param chronon0
+ * @param chronon1
+ * @return
+ * @throws HyracksDataException
+ */
+ abstract protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException;
+
@Override
public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
return new ICopyEvaluatorFactory() {
@@ -62,9 +96,10 @@
return new ICopyEvaluator() {
private DataOutput out = output.getDataOutput();
// one temp. buffer re-used by both children
- private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
- private ICopyEvaluator evalLeft = args[0].createEvaluator(argOut);
- private ICopyEvaluator evalRight = args[1].createEvaluator(argOut);
+ private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
+ private ICopyEvaluator evalLeft = args[0].createEvaluator(argOut0);
+ private ICopyEvaluator evalRight = args[1].createEvaluator(argOut1);
private double[] operandsFloating = new double[args.length];
private long[] operandsInteger = new long[args.length];
private int resultType;
@@ -81,6 +116,12 @@
protected AMutableInt32 aInt32 = new AMutableInt32(0);
protected AMutableInt16 aInt16 = new AMutableInt16((short) 0);
protected AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+
+ protected AMutableDuration aDuration = new AMutableDuration(0, 0);
+ protected AMutableDate aDate = new AMutableDate(0);
+ protected AMutableTime aTime = new AMutableTime(0);
+ protected AMutableDateTime aDatetime = new AMutableDateTime(0);
+
private ATypeTag typeTag;
@SuppressWarnings("rawtypes")
private ISerializerDeserializer serde;
@@ -93,11 +134,16 @@
resultType = 0;
int currentType = 0;
for (int i = 0; i < args.length; i++) {
- argOut.reset();
- if (i == 0)
+ ArrayBackedValueStorage argOut;
+ if (i == 0) {
+ argOut0.reset();
evalLeft.evaluate(tuple);
- else
+ argOut = argOut0;
+ } else {
+ argOut1.reset();
evalRight.evaluate(tuple);
+ argOut = argOut1;
+ }
typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut.getByteArray()[0]);
switch (typeTag) {
case INT8: {
@@ -144,6 +190,14 @@
argOut.getByteArray(), 1);
break;
}
+ case DATE:
+ case TIME:
+ case DATETIME:
+ case DURATION:
+ case YEARMONTHDURATION:
+ case DAYTIMEDURATION:
+ evaluateTemporalArthmeticOperation(typeTag, tuple);
+ return;
case NULL: {
serde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ANULL);
@@ -237,9 +291,236 @@
throw new AlgebricksException(hde);
}
}
+
+ @SuppressWarnings("unchecked")
+ private void evaluateTemporalArthmeticOperation(ATypeTag leftType, IFrameTupleReference tuple)
+ throws HyracksDataException, AlgebricksException {
+ argOut1.reset();
+ evalRight.evaluate(tuple);
+ ATypeTag rightType = EnumDeserializer.ATYPETAGDESERIALIZER
+ .deserialize(argOut1.getByteArray()[0]);
+
+ if (leftType == ATypeTag.NULL || rightType == ATypeTag.NULL) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+ serde.serialize(ANull.NULL, out);
+ return;
+ }
+
+ if (rightType == leftType) {
+
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADURATION);
+
+ long leftChronon = 0, rightChronon = 0, dayTime = 0;
+
+ int yearMonth = 0;
+
+ switch (leftType) {
+ case DATE:
+ leftChronon = ADateSerializerDeserializer.getChronon(argOut0.getByteArray(), 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ rightChronon = ADateSerializerDeserializer.getChronon(argOut1.getByteArray(), 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+
+ break;
+ case TIME:
+ leftChronon = ATimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
+ rightChronon = ATimeSerializerDeserializer.getChronon(argOut1.getByteArray(), 1);
+ break;
+ case DATETIME:
+ leftChronon = ADateTimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
+ rightChronon = ADateTimeSerializerDeserializer
+ .getChronon(argOut1.getByteArray(), 1);
+ break;
+ case YEARMONTHDURATION:
+ yearMonth = (int) evaluateTimeInstanceArithmetic(
+ AYearMonthDurationSerializerDeserializer.getYearMonth(
+ argOut0.getByteArray(), 1),
+ AYearMonthDurationSerializerDeserializer.getYearMonth(
+ argOut1.getByteArray(), 1));
+ break;
+ case DAYTIMEDURATION:
+ leftChronon = ADayTimeDurationSerializerDeserializer.getDayTime(
+ argOut0.getByteArray(), 1);
+ rightChronon = ADayTimeDurationSerializerDeserializer.getDayTime(
+ argOut1.getByteArray(), 1);
+ break;
+ default:
+ throw new NotImplementedException();
+ }
+
+ dayTime = evaluateTimeInstanceArithmetic(leftChronon, rightChronon);
+
+ aDuration.setValue(yearMonth, dayTime);
+
+ serde.serialize(aDuration, out);
+
+ } else {
+ long chronon = 0, dayTime = 0;
+ int yearMonth = 0;
+ ATypeTag resultType = null;
+
+ boolean isTimeOnly = false;
+
+ switch (leftType) {
+ case TIME:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ATIME);
+ chronon = ATimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
+ isTimeOnly = true;
+ resultType = ATypeTag.TIME;
+ switch (rightType) {
+ case DAYTIMEDURATION:
+ dayTime = ADayTimeDurationSerializerDeserializer.getDayTime(
+ argOut1.getByteArray(), 1);
+ break;
+ case DURATION:
+ dayTime = ADurationSerializerDeserializer.getDayTime(
+ argOut1.getByteArray(), 1);
+ yearMonth = ADurationSerializerDeserializer.getYearMonth(
+ argOut1.getByteArray(), 1);
+ break;
+ default:
+ throw new NotImplementedException(getIdentifier().getName()
+ + ": arithmetic operation between " + leftType + " and a "
+ + rightType + " value is not supported.");
+ }
+ break;
+ case DATE:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATE);
+ resultType = ATypeTag.DATE;
+ chronon = ADateSerializerDeserializer.getChronon(argOut0.getByteArray(), 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ case DATETIME:
+ if (leftType == ATypeTag.DATETIME) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATETIME);
+ resultType = ATypeTag.DATETIME;
+ chronon = ADateTimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
+ }
+ switch (rightType) {
+ case DURATION:
+ yearMonth = ADurationSerializerDeserializer.getYearMonth(
+ argOut1.getByteArray(), 1);
+ dayTime = ADurationSerializerDeserializer.getDayTime(
+ argOut1.getByteArray(), 1);
+ break;
+ case YEARMONTHDURATION:
+ yearMonth = AYearMonthDurationSerializerDeserializer.getYearMonth(
+ argOut1.getByteArray(), 1);
+ break;
+ case DAYTIMEDURATION:
+ dayTime = ADayTimeDurationSerializerDeserializer.getDayTime(
+ argOut1.getByteArray(), 1);
+ break;
+ default:
+ throw new NotImplementedException(getIdentifier().getName()
+ + ": arithmetic operation between " + leftType + " and a "
+ + rightType + " value is not supported.");
+ }
+ break;
+ case YEARMONTHDURATION:
+ yearMonth = AYearMonthDurationSerializerDeserializer.getYearMonth(
+ argOut0.getByteArray(), 1);
+ switch (rightType) {
+ case DATETIME:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATETIME);
+ resultType = ATypeTag.DATETIME;
+ chronon = ADateTimeSerializerDeserializer.getChronon(
+ argOut1.getByteArray(), 1);
+ break;
+ case DATE:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATE);
+ resultType = ATypeTag.DATE;
+ chronon = ADateSerializerDeserializer.getChronon(argOut1.getByteArray(), 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ break;
+ default:
+ throw new NotImplementedException(getIdentifier().getName()
+ + ": arithmetic operation between " + leftType + " and a "
+ + rightType + " value is not supported.");
+ }
+ break;
+ case DURATION:
+ yearMonth = ADurationSerializerDeserializer.getYearMonth(argOut0.getByteArray(), 1);
+ dayTime = ADurationSerializerDeserializer.getDayTime(argOut0.getByteArray(), 1);
+ case DAYTIMEDURATION:
+ if (leftType == ATypeTag.DAYTIMEDURATION) {
+ dayTime = ADayTimeDurationSerializerDeserializer.getDayTime(
+ argOut0.getByteArray(), 1);
+ }
+ switch (rightType) {
+ case DATETIME:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATETIME);
+ resultType = ATypeTag.DATETIME;
+ chronon = ADateTimeSerializerDeserializer.getChronon(
+ argOut1.getByteArray(), 1);
+ break;
+ case DATE:
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATE);
+ resultType = ATypeTag.DATE;
+ chronon = ADateSerializerDeserializer.getChronon(argOut1.getByteArray(), 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ break;
+ case TIME:
+ if (yearMonth == 0) {
+ serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ATIME);
+ resultType = ATypeTag.TIME;
+ chronon = ATimeSerializerDeserializer.getChronon(
+ argOut1.getByteArray(), 1);
+ isTimeOnly = true;
+ break;
+ }
+ default:
+ throw new NotImplementedException(getIdentifier().getName()
+ + ": arithmetic operation between " + leftType + " and a "
+ + rightType + " value is not supported.");
+ }
+ break;
+ default:
+ throw new NotImplementedException(getIdentifier().getName()
+ + ": arithmetic operation between " + leftType + " and a " + rightType
+ + " value is not supported.");
+ }
+
+ chronon = evaluateTimeDurationArithmetic(chronon, yearMonth, dayTime, isTimeOnly);
+
+ switch (resultType) {
+ case DATE:
+
+ if (chronon < 0 && chronon % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
+ chronon = chronon / GregorianCalendarSystem.CHRONON_OF_DAY - 1;
+ } else {
+ chronon = chronon / GregorianCalendarSystem.CHRONON_OF_DAY;
+ }
+ aDate.setValue((int) chronon);
+ serde.serialize(aDate, out);
+ break;
+ case TIME:
+ aTime.setValue((int) chronon);
+ serde.serialize(aTime, out);
+ break;
+ case DATETIME:
+ aDatetime.setValue(chronon);
+ serde.serialize(aDatetime, out);
+ break;
+ default:
+ throw new NotImplementedException(getIdentifier().getName()
+ + ": arithmetic operation between " + leftType + " and a " + rightType
+ + " value is not supported.");
+
+ }
+ }
+ }
};
}
};
}
-
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
index bb63d27..7cd96fa 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.asterix.runtime.evaluators.functions;
+import edu.uci.ics.asterix.om.base.temporal.DurationArithmeticOperations;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -49,4 +50,15 @@
protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
return lhs + rhs;
}
+
+ @Override
+ protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+ throws HyracksDataException {
+ return DurationArithmeticOperations.addDuration(chronon, yearMonth, dayTime, isTimeOnly);
+ }
+
+ @Override
+ protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
+ throw new HyracksDataException("Undefined addition operation between two time instances.");
+ }
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
index 208c454..502c013 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
@@ -17,6 +17,7 @@
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -61,4 +62,15 @@
return AsterixBuiltinFunctions.CARET;
}
+ @Override
+ protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+ throws HyracksDataException {
+ throw new NotImplementedException("Caret operation is not defined for temporal types");
+ }
+
+ @Override
+ protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
+ throw new NotImplementedException("Caret operation is not defined for temporal types");
+ }
+
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
index 9b3b0e9..069007c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
@@ -17,6 +17,7 @@
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -45,4 +46,15 @@
protected double evaluateDouble(double lhs, double rhs) {
return lhs / rhs;
}
+
+ @Override
+ protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+ throws HyracksDataException {
+ throw new NotImplementedException("Divide operation is not defined for temporal types");
+ }
+
+ @Override
+ protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
+ throw new NotImplementedException("Divide operation is not defined for temporal types");
+ }
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
index a95c693..92f15b4 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
@@ -17,6 +17,7 @@
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -57,4 +58,15 @@
return lhs * rhs;
}
+
+ @Override
+ protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+ throws HyracksDataException {
+ throw new NotImplementedException("Multiply operation is not defined for temporal types");
+ }
+
+ @Override
+ protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
+ throw new NotImplementedException("Multiply operation is not defined for temporal types");
+ }
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericSubDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericSubDescriptor.java
new file mode 100644
index 0000000..84c308f
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/NumericSubDescriptor.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import edu.uci.ics.asterix.om.base.temporal.DurationArithmeticOperations;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NumericSubDescriptor extends AbstractNumericArithmeticEval {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new NumericSubDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateInteger(long, long)
+ */
+ @Override
+ protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
+ long res = lhs - rhs;
+ if (lhs > 0) {
+ if (rhs < 0 && res < 0)
+ throw new HyracksDataException("Overflow adding " + lhs + " + " + rhs);
+ } else if (rhs > 0 && res > 0)
+ throw new HyracksDataException("Underflow adding " + lhs + " + " + rhs);
+ return res;
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateDouble(double, double)
+ */
+ @Override
+ protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
+ return lhs - rhs;
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateTimeDurationArithmetic(long, int, long, boolean)
+ */
+ @Override
+ protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+ throws HyracksDataException {
+ return DurationArithmeticOperations.addDuration(chronon, -1 * yearMonth, -1 * dayTime, isTimeOnly);
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateTimeInstanceArithmetic(long, long)
+ */
+ @Override
+ protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
+ return evaluateInteger(chronon0, chronon1);
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AddDateDurationDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AddDateDurationDescriptor.java
deleted file mode 100644
index eccee9d..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AddDateDurationDescriptor.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
-
-import java.io.DataOutput;
-
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.ADate;
-import edu.uci.ics.asterix.om.base.AMutableDate;
-import edu.uci.ics.asterix.om.base.ANull;
-import edu.uci.ics.asterix.om.base.temporal.DurationArithmeticOperations;
-import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
-import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
-import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.asterix.om.types.EnumDeserializer;
-import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-public class AddDateDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
- private final static long serialVersionUID = 1L;
- public final static FunctionIdentifier FID = AsterixBuiltinFunctions.ADD_DATE_DURATION;
-
- // allowed input types
- private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
- private final static byte SER_DATE_TYPE_TAG = ATypeTag.DATE.serialize();
- private final static byte SER_DURATION_TYPE_TAG = ATypeTag.DURATION.serialize();
-
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AddDateDurationDescriptor();
- }
- };
-
- @Override
- public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
- return new ICopyEvaluatorFactory() {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
- return new ICopyEvaluator() {
-
- private DataOutput out = output.getDataOutput();
- private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
- private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
- private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
- private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
-
- // possible output types
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ANULL);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADate> dateSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADATE);
-
- private AMutableDate aDate = new AMutableDate(0);
-
- @Override
- public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
- argOut0.reset();
- eval0.evaluate(tuple);
- argOut1.reset();
- eval1.evaluate(tuple);
-
- try {
- if (argOut0.getByteArray()[0] == SER_NULL_TYPE_TAG
- || argOut1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
- nullSerde.serialize(ANull.NULL, out);
- return;
- }
-
- if (argOut0.getByteArray()[0] != SER_DATE_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects type DATE/NULL for parameter 0 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0]));
- }
-
- if (argOut1.getByteArray()[0] != SER_DURATION_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects type DURATION/NULL for parameter 1 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0]));
- }
-
- // get duration fields: yearMonth field and dayTime field
- int yearMonth = ADurationSerializerDeserializer.getYearMonth(argOut1.getByteArray(), 1);
- long dayTime = ADurationSerializerDeserializer.getDayTime(argOut1.getByteArray(), 1);
-
- // get date fields
- long datetimeChronon = ADateSerializerDeserializer.getChronon(argOut0.getByteArray(), 1)
- * GregorianCalendarSystem.CHRONON_OF_DAY;
-
- datetimeChronon = DurationArithmeticOperations.addDuration(datetimeChronon, yearMonth,
- dayTime);
-
- int dateChrononInDays = (int) (datetimeChronon / GregorianCalendarSystem.CHRONON_OF_DAY);
- if (dateChrononInDays < 0 && datetimeChronon % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
- dateChrononInDays -= 1;
- }
-
- aDate.setValue(dateChrononInDays);
-
- dateSerde.serialize(aDate, out);
-
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
- }
- }
- };
- }
- };
- }
-
- @Override
- public FunctionIdentifier getIdentifier() {
- return FID;
- }
-
-}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AddDatetimeDurationDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AddDatetimeDurationDescriptor.java
deleted file mode 100644
index 0037b38..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AddDatetimeDurationDescriptor.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
-
-import java.io.DataOutput;
-
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.ADateTime;
-import edu.uci.ics.asterix.om.base.AMutableDateTime;
-import edu.uci.ics.asterix.om.base.ANull;
-import edu.uci.ics.asterix.om.base.temporal.DurationArithmeticOperations;
-import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
-import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.asterix.om.types.EnumDeserializer;
-import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-public class AddDatetimeDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
- private final static long serialVersionUID = 1L;
- public final static FunctionIdentifier FID = AsterixBuiltinFunctions.ADD_DATETIME_DURATION;
-
- // allowed input types
- private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
- private final static byte SER_DATETIME_TYPE_TAG = ATypeTag.DATETIME.serialize();
- private final static byte SER_DURATION_TYPE_TAG = ATypeTag.DURATION.serialize();
-
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AddDatetimeDurationDescriptor();
- }
- };
-
- /* (non-Javadoc)
- * @see edu.uci.ics.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
- */
- @Override
- public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
- return new ICopyEvaluatorFactory() {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
- return new ICopyEvaluator() {
-
- private DataOutput out = output.getDataOutput();
- private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
- private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
- private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
- private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
-
- // possible output types
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ANULL);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADateTime> datetimeSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADATETIME);
-
- private AMutableDateTime aDatetime = new AMutableDateTime(0);
-
- @Override
- public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
- argOut0.reset();
- eval0.evaluate(tuple);
- argOut1.reset();
- eval1.evaluate(tuple);
-
- try {
- if (argOut0.getByteArray()[0] == SER_NULL_TYPE_TAG
- || argOut1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
- nullSerde.serialize(ANull.NULL, out);
- return;
- }
-
- if (argOut0.getByteArray()[0] != SER_DATETIME_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects type DATETIME/NULL for parameter 0 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0]));
- }
-
- if (argOut1.getByteArray()[0] != SER_DURATION_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects type DURATION/NULL for parameter 1 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0]));
- }
-
- // get duration fields: yearMonth field and dayTime field
- int yearMonth = ADurationSerializerDeserializer.getYearMonth(argOut1.getByteArray(), 1);
- long dayTime = ADurationSerializerDeserializer.getDayTime(argOut1.getByteArray(), 1);
-
- // get date fields
- long datetimeChronon = ADateTimeSerializerDeserializer
- .getChronon(argOut0.getByteArray(), 1);
-
- datetimeChronon = DurationArithmeticOperations.addDuration(datetimeChronon, yearMonth,
- dayTime);
-
- aDatetime.setValue(datetimeChronon);
-
- datetimeSerde.serialize(aDatetime, out);
-
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
- }
- }
- };
- }
- };
- }
-
- /* (non-Javadoc)
- * @see edu.uci.ics.asterix.om.functions.IFunctionDescriptor#getIdentifier()
- */
- @Override
- public FunctionIdentifier getIdentifier() {
- return FID;
- }
-
-}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
index 9e603ed..9ef3f58 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
@@ -127,7 +127,7 @@
* GregorianCalendarSystem.CHRONON_OF_DAY;
long endingTimePoint = DurationArithmeticOperations.addDuration(startingTimePoint,
- yearMonthDurationInMonths, dayTimeDurationInMs);
+ yearMonthDurationInMonths, dayTimeDurationInMs, false);
if (startingTimePoint == endingTimePoint) {
aDuration.setValue(0, 0);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
index 10cc1c0..da5f195 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
@@ -141,7 +141,7 @@
1);
long endingTimePoint = DurationArithmeticOperations.addDuration(startingTimePoint,
- yearMonthDurationInMonths, dayTimeDurationInMs);
+ yearMonthDurationInMonths, dayTimeDurationInMs, false);
if (startingTimePoint == endingTimePoint) {
aDuration.setValue(0, 0);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
new file mode 100644
index 0000000..11ba264
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
@@ -0,0 +1,150 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class DayOfWeekDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.DAY_OF_WEEK;
+
+ // allowed input types
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static byte SER_DATETIME_TYPE_TAG = ATypeTag.DATETIME.serialize();
+ private final static byte SER_DATE_TYPE_TAG = ATypeTag.DATE.serialize();
+
+ // Fixed week day anchor: Thursday, 1 January 1970
+ private final static int ANCHOR_WEEKDAY = 4;
+
+ public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new DayOfWeekDescriptor();
+ }
+
+ };
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ return new ICopyEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new ICopyEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+ private ICopyEvaluator eval = args[0].createEvaluator(argOut);
+
+ // possible returning types
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+ private AMutableInt32 aInt32 = new AMutableInt32(0);
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut.reset();
+ eval.evaluate(tuple);
+ try {
+ if (argOut.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+ nullSerde.serialize(ANull.NULL, out);
+ } else {
+ int daysSinceAnchor;
+ int reminder = 0;
+ if (argOut.getByteArray()[0] == SER_DATETIME_TYPE_TAG) {
+ daysSinceAnchor = (int) (ADateTimeSerializerDeserializer.getChronon(
+ argOut.getByteArray(), 1) / GregorianCalendarSystem.CHRONON_OF_DAY);
+ reminder = (int) (ADateTimeSerializerDeserializer.getChronon(argOut.getByteArray(),
+ 1) % GregorianCalendarSystem.CHRONON_OF_DAY);
+ } else if (argOut.getByteArray()[0] == SER_DATE_TYPE_TAG) {
+ daysSinceAnchor = ADateSerializerDeserializer.getChronon(argOut.getByteArray(), 1);
+ } else {
+ throw new AlgebricksException(
+ FID.getName()
+ + ": expects input type DATETIME/DATE/NULL but got "
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut
+ .getByteArray()[0]));
+ }
+
+ // adjust the day before 1970-01-01
+ if (daysSinceAnchor < 0 && reminder != 0) {
+ daysSinceAnchor -= 1;
+ }
+
+ // compute the weekday (0-based, and 0 = Sunday). Adjustment is needed as the anchor day is Thursday
+ int weekday = (daysSinceAnchor + ANCHOR_WEEKDAY) % 7;
+
+ // handle the negative weekday
+ if (weekday < 0) {
+ weekday += 7;
+ }
+
+ // convert from 0-based to 1-based (so 7 = Sunday)
+ if (weekday == 0) {
+ weekday = 7;
+ }
+
+ aInt32.setValue(weekday);
+
+ int32Serde.serialize(aInt32, out);
+ }
+ } catch (HyracksDataException hex) {
+ throw new AlgebricksException(hex);
+ }
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
new file mode 100644
index 0000000..98c1d16
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
@@ -0,0 +1,282 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADayTimeDurationSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AInterval;
+import edu.uci.ics.asterix.om.base.AMutableInterval;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.DurationArithmeticOperations;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class IntervalBinDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new IntervalBinDescriptor();
+ }
+ };
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ return new ICopyEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new ICopyEvaluator() {
+
+ private final DataOutput out = output.getDataOutput();
+
+ private final ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
+ private final ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
+ private final ArrayBackedValueStorage argOut2 = new ArrayBackedValueStorage();
+
+ private final ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
+ private final ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
+ private final ICopyEvaluator eval2 = args[2].createEvaluator(argOut2);
+
+ private final AMutableInterval aInterval = new AMutableInterval(0, 0, (byte) -1);
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInterval> intervalSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINTERVAL);
+
+ private final GregorianCalendarSystem GREG_CAL = GregorianCalendarSystem.getInstance();
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut0.reset();
+ eval0.evaluate(tuple);
+
+ ATypeTag type0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0]);
+
+ long chrononToBin = 0;
+ switch (type0) {
+ case DATE:
+ chrononToBin = ADateSerializerDeserializer.getChronon(argOut0.getByteArray(), 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ break;
+ case TIME:
+ chrononToBin = ATimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
+ break;
+ case DATETIME:
+ chrononToBin = ADateTimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
+ break;
+ case NULL:
+ try {
+ nullSerde.serialize(ANull.NULL, out);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ return;
+ default:
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": the first argument should be DATE/TIME/DATETIME/NULL but got " + type0);
+
+ }
+
+ argOut1.reset();
+ eval1.evaluate(tuple);
+
+ ATypeTag type1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0]);
+
+ if (type0 != type1) {
+ if (type0 != ATypeTag.NULL && type1 != ATypeTag.NULL)
+ throw new AlgebricksException(getIdentifier().getName() + ": expecting " + type0
+ + " for the second argument but got " + type1);
+ }
+
+ long chrononToStart = 0;
+ switch (type1) {
+ case DATE:
+ chrononToStart = ADateSerializerDeserializer.getChronon(argOut1.getByteArray(), 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ break;
+ case TIME:
+ chrononToStart = ATimeSerializerDeserializer.getChronon(argOut1.getByteArray(), 1);
+ break;
+ case DATETIME:
+ chrononToStart = ADateTimeSerializerDeserializer.getChronon(argOut1.getByteArray(), 1);
+ break;
+ case NULL:
+ try {
+ nullSerde.serialize(ANull.NULL, out);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ return;
+ default:
+ throw new AlgebricksException(getIdentifier().getName() + ": expecting " + type0
+ + " for the second argument but got " + type1);
+ }
+
+ argOut2.reset();
+ eval2.evaluate(tuple);
+
+ ATypeTag type2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut2.getByteArray()[0]);
+
+ int yearMonth = 0;
+ long dayTime = 0;
+ long binIndex, binStartChronon, binEndChronon;
+ switch (type2) {
+ case YEARMONTHDURATION:
+
+ yearMonth = AYearMonthDurationSerializerDeserializer.getYearMonth(
+ argOut2.getByteArray(), 1);
+
+ int yearStart = GREG_CAL.getYear(chrononToStart);
+ int monthStart = GREG_CAL.getMonthOfYear(chrononToStart, yearStart);
+ int yearToBin = GREG_CAL.getYear(chrononToBin);
+ int monthToBin = GREG_CAL.getMonthOfYear(chrononToBin, yearToBin);
+
+ int totalMonths = (yearToBin - yearStart) * 12 + (monthToBin - monthStart);
+
+ binIndex = totalMonths / yearMonth
+ + ((totalMonths < 0 && totalMonths % yearMonth != 0) ? -1 : 0);
+
+ if (binIndex > Integer.MAX_VALUE) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": Overflowing time value to be binned!");
+ }
+
+ if (binIndex < Integer.MIN_VALUE) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": Underflowing time value to be binned!");
+ }
+
+ break;
+
+ case DAYTIMEDURATION:
+ dayTime = ADayTimeDurationSerializerDeserializer.getDayTime(argOut2.getByteArray(), 1);
+
+ long totalChronon = chrononToBin - chrononToStart;
+
+ binIndex = totalChronon / dayTime
+ + ((totalChronon < 0 && totalChronon % dayTime != 0) ? -1 : 0);
+
+ break;
+ case NULL:
+ try {
+ nullSerde.serialize(ANull.NULL, out);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ return;
+ default:
+ throw new AlgebricksException(
+ getIdentifier().getName()
+ + ": expecting YEARMONTHDURATION/DAYTIMEDURATION for the thrid argument but got "
+ + type2);
+ }
+
+ switch (type0) {
+ case DATE:
+ binStartChronon = DurationArithmeticOperations.addDuration(chrononToStart, yearMonth
+ * (int) binIndex, dayTime * binIndex, false);
+ binEndChronon = DurationArithmeticOperations.addDuration(chrononToStart, yearMonth
+ * ((int) binIndex + 1), dayTime * (binIndex + 1), false);
+
+ binStartChronon = binStartChronon
+ / GregorianCalendarSystem.CHRONON_OF_DAY
+ + ((binStartChronon < 0 && binStartChronon
+ % GregorianCalendarSystem.CHRONON_OF_DAY != 0) ? -1 : 0);
+ binEndChronon = binEndChronon
+ / GregorianCalendarSystem.CHRONON_OF_DAY
+ + ((binEndChronon < 0 && binEndChronon % GregorianCalendarSystem.CHRONON_OF_DAY != 0) ? -1
+ : 0);
+ break;
+ case TIME:
+ if (yearMonth != 0) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": cannot create year-month bin for a time value");
+ }
+ binStartChronon = DurationArithmeticOperations.addDuration(chrononToStart, yearMonth
+ * (int) binIndex, dayTime * binIndex, true);
+ binEndChronon = DurationArithmeticOperations.addDuration(chrononToStart, yearMonth
+ * ((int) binIndex + 1), dayTime * (binIndex + 1), true);
+ break;
+ case DATETIME:
+ binStartChronon = DurationArithmeticOperations.addDuration(chrononToStart, yearMonth
+ * (int) binIndex, dayTime * binIndex, false);
+ binEndChronon = DurationArithmeticOperations.addDuration(chrononToStart, yearMonth
+ * ((int) binIndex + 1), dayTime * (binIndex + 1), false);
+ break;
+ case NULL:
+ try {
+ nullSerde.serialize(ANull.NULL, out);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ return;
+ default:
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": the first argument should be DATE/TIME/DATETIME/NULL but got " + type0);
+
+ }
+ aInterval.setValue(binStartChronon, binEndChronon, type0.serialize());
+ try {
+ intervalSerde.serialize(aInterval, out);
+ return;
+ } catch (HyracksDataException ex) {
+ throw new AlgebricksException(ex);
+ }
+
+ }
+ };
+ }
+
+ };
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return AsterixBuiltinFunctions.INTERVAL_BIN;
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
index 9a66d45..d2f2b07 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
@@ -16,7 +16,7 @@
import java.io.DataOutput;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.om.base.AInt32;
import edu.uci.ics.asterix.om.base.AMutableInt32;
@@ -96,7 +96,7 @@
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0]));
}
- aInt32.setValue(AYearMonthDurationerializerDeserializer.getYearMonth(
+ aInt32.setValue(AYearMonthDurationSerializerDeserializer.getYearMonth(
argOut0.getByteArray(), 1));
int32Serde.serialize(aInt32, out);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
new file mode 100644
index 0000000..782d424
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
@@ -0,0 +1,168 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ADate;
+import edu.uci.ics.asterix.om.base.AMutableDate;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <b>|(bar)</b> is a special separator used to separate different formatting options.
+ * Multiple format strings can be used by separating them using <b>|(bar)</b>, and the parsing will be successful only when the format string has the <b>exact</b> match with the given data string. This means that a time string like <it>08:23:12 AM</it> will not be valid for the format string <it>h:m:s</it> as there is no AM/PM format character in the format string.
+ * <p/>
+ */
+public class ParseDateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PARSE_DATE;
+
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
+
+ public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new ParseDateDescriptor();
+ }
+ };
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ return new ICopyEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new ICopyEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
+ private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
+ private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ADate> dateSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATE);
+
+ private AMutableDate aDate = new AMutableDate(0);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut0.reset();
+ eval0.evaluate(tuple);
+ argOut1.reset();
+ eval1.evaluate(tuple);
+
+ try {
+ if (argOut0.getByteArray()[0] == SER_NULL_TYPE_TAG
+ || argOut1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+ nullSerde.serialize(ANull.NULL, out);
+ return;
+ }
+
+ if (argOut0.getByteArray()[0] != SER_STRING_TYPE_TAG
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects two strings but got ("
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ + ", "
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
+ + ")");
+ }
+ int length0 = (argOut0.getByteArray()[1] & 0xff << 8)
+ + (argOut0.getByteArray()[2] & 0xff << 0);
+ int length1 = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ long chronon = 0;
+
+ int formatStart = 3;
+ int formatLength = 0;
+ boolean processSuccessfully = false;
+ while (!processSuccessfully && formatStart < 3 + length1) {
+ // search for "|"
+ formatLength = 0;
+ for (; formatStart + formatLength < 3 + length1; formatLength++) {
+ if (argOut1.getByteArray()[formatStart + formatLength] == '|') {
+ break;
+ }
+ }
+ try {
+ chronon = DT_UTILS.parseDateTime(argOut0.getByteArray(), 3, length0,
+ argOut1.getByteArray(), formatStart, formatLength,
+ DateTimeParseMode.DATE_ONLY);
+ } catch (AsterixTemporalTypeParseException ex) {
+ formatStart += formatLength + 1;
+ continue;
+ }
+ processSuccessfully = true;
+ }
+
+ if (!processSuccessfully) {
+ throw new HyracksDataException(
+ "parse-date: Failed to match with any given format string!");
+ }
+
+ aDate.setValue((int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY));
+ dateSerde.serialize(aDate, out);
+
+ } catch (HyracksDataException ex) {
+ throw new AlgebricksException(ex);
+ }
+ }
+ };
+ }
+
+ };
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
new file mode 100644
index 0000000..89cfbe5
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
@@ -0,0 +1,162 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ADateTime;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ParseDateTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PARSE_DATETIME;
+
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
+
+ public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new ParseDateTimeDescriptor();
+ }
+ };
+
+ @Override
+ public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+ return new ICopyEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new ICopyEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage argOut0 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage argOut1 = new ArrayBackedValueStorage();
+ private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
+ private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
+
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ANULL);
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<ADateTime> datetimeSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ADATETIME);
+
+ private AMutableDateTime aDateTime = new AMutableDateTime(0);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut0.reset();
+ eval0.evaluate(tuple);
+ argOut1.reset();
+ eval1.evaluate(tuple);
+
+ try {
+ if (argOut0.getByteArray()[0] == SER_NULL_TYPE_TAG
+ || argOut1.getByteArray()[0] == SER_NULL_TYPE_TAG) {
+ nullSerde.serialize(ANull.NULL, out);
+ return;
+ }
+
+ if (argOut0.getByteArray()[0] != SER_STRING_TYPE_TAG
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects two strings but got ("
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ + ", "
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
+ + ")");
+ }
+ int length0 = (argOut0.getByteArray()[1] & 0xff << 8)
+ + (argOut0.getByteArray()[2] & 0xff << 0);
+ int length1 = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ long chronon = 0;
+
+ int formatStart = 3;
+ int formatLength = 0;
+ boolean processSuccessfully = false;
+ while (!processSuccessfully && formatStart < 3 + length1) {
+ // search for "|"
+ formatLength = 0;
+ for (; formatStart + formatLength < 3 + length1; formatLength++) {
+ if (argOut1.getByteArray()[formatStart + formatLength] == '|') {
+ break;
+ }
+ }
+ try {
+ chronon = DT_UTILS.parseDateTime(argOut0.getByteArray(), 3, length0,
+ argOut1.getByteArray(), formatStart, formatLength,
+ DateTimeParseMode.DATETIME);
+ } catch (AsterixTemporalTypeParseException ex) {
+ formatStart += formatLength + 1;
+ continue;
+ }
+ processSuccessfully = true;
+ }
+
+ if (!processSuccessfully) {
+ throw new HyracksDataException(
+ "parse-date: Failed to match with any given format string!");
+ }
+
+ aDateTime.setValue(chronon);
+ datetimeSerde.serialize(aDateTime, out);
+
+ } catch (HyracksDataException ex) {
+ throw new AlgebricksException(ex);
+ }
+ }
+ };
+ }
+
+ };
+ }
+
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AddTimeDurationDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
similarity index 60%
rename from asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AddTimeDurationDescriptor.java
rename to asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
index ca86379..8ee76e7 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/AddTimeDurationDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
@@ -16,13 +16,13 @@
import java.io.DataOutput;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
import edu.uci.ics.asterix.om.base.AMutableTime;
import edu.uci.ics.asterix.om.base.ANull;
import edu.uci.ics.asterix.om.base.ATime;
-import edu.uci.ics.asterix.om.base.temporal.DurationArithmeticOperations;
+import edu.uci.ics.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -40,27 +40,23 @@
import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-public class AddTimeDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class ParseTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- private final static long serialVersionUID = 1L;
- public final static FunctionIdentifier FID = AsterixBuiltinFunctions.ADD_TIME_DURATION;
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PARSE_TIME;
- // allowed input types
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
- private final static byte SER_TIME_TYPE_TAG = ATypeTag.TIME.serialize();
- private final static byte SER_DURATION_TYPE_TAG = ATypeTag.DURATION.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@Override
public IFunctionDescriptor createFunctionDescriptor() {
- return new AddTimeDurationDescriptor();
+ return new ParseTimeDescriptor();
}
};
- /* (non-Javadoc)
- * @see edu.uci.ics.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
- */
@Override
public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
return new ICopyEvaluatorFactory() {
@@ -77,7 +73,6 @@
private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
- // possible output types
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ANULL);
@@ -101,49 +96,63 @@
return;
}
- if (argOut0.getByteArray()[0] != SER_TIME_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects type TIME/NULL for parameter 0 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0]));
+ if (argOut0.getByteArray()[0] != SER_STRING_TYPE_TAG
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects two strings but got ("
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ + ", "
+ + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
+ + ")");
+ }
+ int length0 = (argOut0.getByteArray()[1] & 0xff << 8)
+ + (argOut0.getByteArray()[2] & 0xff << 0);
+ int length1 = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ long chronon = 0;
+
+ int formatStart = 3;
+ int formatLength = 0;
+ boolean processSuccessfully = false;
+ while (!processSuccessfully && formatStart < 3 + length1) {
+ // search for "|"
+ formatLength = 0;
+ for (; formatStart + formatLength < 3 + length1; formatLength++) {
+ if (argOut1.getByteArray()[formatStart + formatLength] == '|') {
+ break;
+ }
+ }
+ try {
+ chronon = DT_UTILS.parseDateTime(argOut0.getByteArray(), 3, length0,
+ argOut1.getByteArray(), formatStart, formatLength,
+ DateTimeParseMode.TIME_ONLY);
+ } catch (AsterixTemporalTypeParseException ex) {
+ formatStart += formatLength + 1;
+ continue;
+ }
+ processSuccessfully = true;
}
- if (argOut1.getByteArray()[0] != SER_DURATION_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects type DURATION/NULL for parameter 1 but got "
- + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0]));
+ if (!processSuccessfully) {
+ throw new HyracksDataException(
+ "parse-date: Failed to match with any given format string!");
}
- // get duration fields: yearMonth field and dayTime field
- int yearMonth = ADurationSerializerDeserializer.getYearMonth(argOut1.getByteArray(), 1);
-
- // cannot add a year-month duration to a time value
- if (yearMonth != 0) {
- throw new AlgebricksException(FID.getName()
- + ": a TIME value cannot be added by a DURATION value with year-month field.");
- }
-
- long dayTime = ADurationSerializerDeserializer.getDayTime(argOut1.getByteArray(), 1);
-
- // get time fields
- int timeChronon = ATimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
-
- timeChronon = DurationArithmeticOperations.addDuration(timeChronon, dayTime);
-
- aTime.setValue(timeChronon);
-
+ aTime.setValue((int) chronon);
timeSerde.serialize(aTime, out);
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ } catch (HyracksDataException ex) {
+ throw new AlgebricksException(ex);
}
}
};
}
+
};
}
/* (non-Javadoc)
- * @see edu.uci.ics.asterix.om.functions.IFunctionDescriptor#getIdentifier()
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
*/
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/SubtractDateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
similarity index 71%
rename from asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/SubtractDateDescriptor.java
rename to asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
index eaaf900..a48e44c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/SubtractDateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
@@ -15,13 +15,14 @@
package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
import java.io.DataOutput;
+import java.io.IOException;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.ADuration;
-import edu.uci.ics.asterix.om.base.AMutableDuration;
import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -34,25 +35,25 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-public class SubtractDateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class PrintDateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- private final static long serialVersionUID = 1L;
- public final static FunctionIdentifier FID = AsterixBuiltinFunctions.SUBTRACT_DATE;
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PRINT_DATE;
- // allowed input types
- private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
private final static byte SER_DATE_TYPE_TAG = ATypeTag.DATE.serialize();
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@Override
public IFunctionDescriptor createFunctionDescriptor() {
- return new SubtractDateDescriptor();
+ return new PrintDateDescriptor();
}
};
@@ -72,15 +73,11 @@
private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
- // possible output types
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ANULL);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADuration> durationSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADURATION);
- private AMutableDuration aDuration = new AMutableDuration(0, 0);
+ private StringBuilder sbder = new StringBuilder();
@Override
public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
@@ -97,31 +94,39 @@
}
if (argOut0.getByteArray()[0] != SER_DATE_TYPE_TAG
- && argOut1.getByteArray()[0] != SER_DATE_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects input type (DATE/NULL, DATE/NULL) but got ("
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects (DATE, STRING) but got ("
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ ", "
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
- + ").");
+ + ")");
}
- long durationChronon = (ADateSerializerDeserializer.getChronon(argOut0.getByteArray(), 1) - ADateSerializerDeserializer
- .getChronon(argOut1.getByteArray(), 1)) * GregorianCalendarSystem.CHRONON_OF_DAY;
+ long chronon = ADateSerializerDeserializer.getChronon(argOut0.getByteArray(), 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ int formatLength = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ sbder.delete(0, sbder.length());
+ DT_UTILS.printDateTime(chronon, 0, argOut1.getByteArray(), 3, formatLength, sbder,
+ DateTimeParseMode.DATE_ONLY);
- aDuration.setValue(0, durationChronon);
+ out.writeByte(ATypeTag.STRING.serialize());
+ out.writeUTF(sbder.toString());
- durationSerde.serialize(aDuration, out);
-
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ } catch (IOException ex) {
+ throw new AlgebricksException(ex);
}
}
};
}
+
};
}
+ /* (non-Javadoc)
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
@Override
public FunctionIdentifier getIdentifier() {
return FID;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/SubtractDatetimeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
similarity index 71%
rename from asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/SubtractDatetimeDescriptor.java
rename to asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
index b43892d..9c507e3 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/SubtractDatetimeDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
@@ -15,12 +15,13 @@
package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
import java.io.DataOutput;
+import java.io.IOException;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.ADuration;
-import edu.uci.ics.asterix.om.base.AMutableDuration;
import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -33,31 +34,28 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-public class SubtractDatetimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class PrintDateTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- private final static long serialVersionUID = 1L;
- public final static FunctionIdentifier FID = AsterixBuiltinFunctions.SUBTRACT_DATETIME;
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PRINT_DATETIME;
- // allowed input types
- private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
private final static byte SER_DATETIME_TYPE_TAG = ATypeTag.DATETIME.serialize();
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@Override
public IFunctionDescriptor createFunctionDescriptor() {
- return new SubtractDatetimeDescriptor();
+ return new PrintDateTimeDescriptor();
}
};
- /* (non-Javadoc)
- * @see edu.uci.ics.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
- */
@Override
public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
return new ICopyEvaluatorFactory() {
@@ -74,15 +72,11 @@
private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
- // possible output types
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ANULL);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADuration> durationSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADURATION);
- private AMutableDuration aDuration = new AMutableDuration(0, 0);
+ private StringBuilder sbder = new StringBuilder();
@Override
public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
@@ -99,34 +93,36 @@
}
if (argOut0.getByteArray()[0] != SER_DATETIME_TYPE_TAG
- && argOut1.getByteArray()[0] != SER_DATETIME_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects input type (DATETIME/NULL, DATETIME/NULL) but got ("
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects (DATETIME, STRING) but got ("
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ ", "
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
- + ").");
+ + ")");
}
+ long chronon = ADateTimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
+ int formatLength = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ sbder.delete(0, sbder.length());
+ DT_UTILS.printDateTime(chronon, 0, argOut1.getByteArray(), 3, formatLength, sbder,
+ DateTimeParseMode.DATETIME);
- long durationChronon = ADateTimeSerializerDeserializer
- .getChronon(argOut0.getByteArray(), 1)
- - ADateTimeSerializerDeserializer.getChronon(argOut1.getByteArray(), 1);
+ out.writeByte(ATypeTag.STRING.serialize());
+ out.writeUTF(sbder.toString());
- aDuration.setValue(0, durationChronon);
-
- durationSerde.serialize(aDuration, out);
-
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ } catch (IOException ex) {
+ throw new AlgebricksException(ex);
}
}
};
}
+
};
}
/* (non-Javadoc)
- * @see edu.uci.ics.asterix.om.functions.IFunctionDescriptor#getIdentifier()
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
*/
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/SubtractTimeDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
similarity index 71%
rename from asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/SubtractTimeDescriptor.java
rename to asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
index 24edca6..7d589a4 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/SubtractTimeDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
@@ -15,12 +15,13 @@
package edu.uci.ics.asterix.runtime.evaluators.functions.temporal;
import java.io.DataOutput;
+import java.io.IOException;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.ADuration;
-import edu.uci.ics.asterix.om.base.AMutableDuration;
import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils;
+import edu.uci.ics.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -33,31 +34,28 @@
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-public class SubtractTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class PrintTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- private final static long serialVersionUID = 1L;
- public final static FunctionIdentifier FID = AsterixBuiltinFunctions.SUBTRACT_TIME;
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = AsterixBuiltinFunctions.PRINT_TIME;
- // allowed input types
- private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
private final static byte SER_TIME_TYPE_TAG = ATypeTag.TIME.serialize();
+ private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+ private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+ private final static DateTimeFormatUtils DT_UTILS = DateTimeFormatUtils.getInstance();
public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@Override
public IFunctionDescriptor createFunctionDescriptor() {
- return new SubtractTimeDescriptor();
+ return new PrintTimeDescriptor();
}
};
- /* (non-Javadoc)
- * @see edu.uci.ics.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
- */
@Override
public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
return new ICopyEvaluatorFactory() {
@@ -74,15 +72,11 @@
private ICopyEvaluator eval0 = args[0].createEvaluator(argOut0);
private ICopyEvaluator eval1 = args[1].createEvaluator(argOut1);
- // possible output types
@SuppressWarnings("unchecked")
private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ANULL);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADuration> durationSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADURATION);
- private AMutableDuration aDuration = new AMutableDuration(0, 0);
+ private StringBuilder sbder = new StringBuilder();
@Override
public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
@@ -99,33 +93,37 @@
}
if (argOut0.getByteArray()[0] != SER_TIME_TYPE_TAG
- && argOut1.getByteArray()[0] != SER_TIME_TYPE_TAG) {
- throw new AlgebricksException(FID.getName()
- + ": expects input type (TIME/NULL, TIME/NULL) but got ("
+ || argOut1.getByteArray()[0] != SER_STRING_TYPE_TAG) {
+ throw new AlgebricksException(getIdentifier().getName()
+ + ": expects (TIME, STRING) but got ("
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0])
+ ", "
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0])
- + ").");
+ + ")");
}
- int durationChronon = ATimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1)
- - ATimeSerializerDeserializer.getChronon(argOut1.getByteArray(), 1);
+ long chronon = ATimeSerializerDeserializer.getChronon(argOut0.getByteArray(), 1);
+ int formatLength = (argOut1.getByteArray()[1] & 0xff << 8)
+ + (argOut1.getByteArray()[2] & 0xff << 0);
+ sbder.delete(0, sbder.length());
+ DT_UTILS.printDateTime(chronon, 0, argOut1.getByteArray(), 3, formatLength, sbder,
+ DateTimeParseMode.TIME_ONLY);
- aDuration.setValue(0, durationChronon);
+ out.writeByte(ATypeTag.STRING.serialize());
+ out.writeUTF(sbder.toString());
- durationSerde.serialize(aDuration, out);
-
- } catch (HyracksDataException hex) {
- throw new AlgebricksException(hex);
+ } catch (IOException ex) {
+ throw new AlgebricksException(ex);
}
}
};
}
+
};
}
/* (non-Javadoc)
- * @see edu.uci.ics.asterix.om.functions.IFunctionDescriptor#getIdentifier()
+ * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
*/
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
index 25737cd..011836a 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -174,6 +174,7 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.NumericRoundDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.NumericRoundHalfToEven2Descriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.NumericRoundHalfToEvenDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.NumericSubDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.NumericSubtractDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.NumericUnaryMinusDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.OpenRecordConstructorDescriptor;
@@ -211,9 +212,6 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.SwitchCaseDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.UnorderedListConstructorDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.WordTokensDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddDateDurationDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddDatetimeDurationDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddTimeDurationDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AdjustDateTimeForTimeZoneDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AdjustTimeForTimeZoneDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.CalendarDuartionFromDateDescriptor;
@@ -225,6 +223,7 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.DateFromUnixTimeInDaysDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.DatetimeFromDateAndTimeDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.DatetimeFromUnixTimeInMsDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.DayOfWeekDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.DayTimeDurationComparatorDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.DurationEqualDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.DurationFromMillisecondsDescriptor;
@@ -233,6 +232,7 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.GetYearMonthDurationDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalAfterDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalBeforeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalBinDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalCoveredByDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalCoversDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalEndedByDescriptor;
@@ -246,12 +246,12 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalOverlapsDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalStartedByDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalStartsDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MillisecondsFromDayTimeDurationDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MonthsFromYearMonthDurationDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.OverlapDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractDateDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractDatetimeDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractTimeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.ParseDateDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.ParseDateTimeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.ParseTimeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.PrintDateDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.PrintDateTimeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.PrintTimeDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.TimeFromDatetimeDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.TimeFromUnixTimeInMsDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.YearMonthDurationComparatorDecriptor;
@@ -363,7 +363,7 @@
temp.add(NumericAddDescriptor.FACTORY);
temp.add(NumericDivideDescriptor.FACTORY);
temp.add(NumericMultiplyDescriptor.FACTORY);
- temp.add(NumericSubtractDescriptor.FACTORY);
+ temp.add(NumericSubDescriptor.FACTORY);
temp.add(NumericModuloDescriptor.FACTORY);
temp.add(NumericCaretDescriptor.FACTORY);
temp.add(IsNullDescriptor.FACTORY);
@@ -512,16 +512,10 @@
// Temporal functions
temp.add(DateFromUnixTimeInDaysDescriptor.FACTORY);
temp.add(DateFromDatetimeDescriptor.FACTORY);
- temp.add(AddDateDurationDescriptor.FACTORY);
- temp.add(SubtractDateDescriptor.FACTORY);
temp.add(TimeFromUnixTimeInMsDescriptor.FACTORY);
temp.add(TimeFromDatetimeDescriptor.FACTORY);
- temp.add(SubtractTimeDescriptor.FACTORY);
- temp.add(AddTimeDurationDescriptor.FACTORY);
temp.add(DatetimeFromUnixTimeInMsDescriptor.FACTORY);
temp.add(DatetimeFromDateAndTimeDescriptor.FACTORY);
- temp.add(SubtractDatetimeDescriptor.FACTORY);
- temp.add(AddDatetimeDurationDescriptor.FACTORY);
temp.add(CalendarDurationFromDateTimeDescriptor.FACTORY);
temp.add(CalendarDuartionFromDateDescriptor.FACTORY);
temp.add(AdjustDateTimeForTimeZoneDescriptor.FACTORY);
@@ -553,6 +547,14 @@
temp.add(DurationEqualDescriptor.FACTORY);
temp.add(GetYearMonthDurationDescriptor.FACTORY);
temp.add(GetDayTimeDurationDescriptor.FACTORY);
+ temp.add(IntervalBinDescriptor.FACTORY);
+ temp.add(DayOfWeekDescriptor.FACTORY);
+ temp.add(ParseDateDescriptor.FACTORY);
+ temp.add(ParseTimeDescriptor.FACTORY);
+ temp.add(ParseDateTimeDescriptor.FACTORY);
+ temp.add(PrintDateDescriptor.FACTORY);
+ temp.add(PrintTimeDescriptor.FACTORY);
+ temp.add(PrintDateTimeDescriptor.FACTORY);
// Interval constructor
temp.add(AIntervalFromDateConstructorDescriptor.FACTORY);
@@ -959,9 +961,9 @@
return AqlBinaryHashFunctionFamilyProvider.INSTANCE;
}
- @Override
- public IPredicateEvaluatorFactoryProvider getPredicateEvaluatorFactoryProvider() {
- return AqlPredicateEvaluatorFactoryProvider.INSTANCE;
- }
+ @Override
+ public IPredicateEvaluatorFactoryProvider getPredicateEvaluatorFactoryProvider() {
+ return AqlPredicateEvaluatorFactoryProvider.INSTANCE;
+ }
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
index 8092ecd..4f4eba2 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
@@ -18,7 +18,6 @@
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.transactions.DatasetId;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
@@ -50,9 +49,8 @@
try {
ITransactionManager txnManager = ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext()
.getApplicationObject()).getTransactionSubsystem().getTransactionManager();
- ITransactionContext txnContext = txnManager.getTransactionContext(jobId);
- txnContext.setTransactionType(transactionalWrite ? TransactionType.READ_WRITE
- : TransactionType.READ);
+ ITransactionContext txnContext = txnManager.getTransactionContext(jobId, false);
+ txnContext.setWriteTxn(transactionalWrite);
txnManager.completedTransaction(txnContext, new DatasetId(-1), -1,
!(jobStatus == JobStatus.FAILURE));
} catch (ACIDException e) {
@@ -64,7 +62,7 @@
public void jobletStart() {
try {
((IAsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
- .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId);
+ .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId, true);
} catch (ACIDException e) {
throw new Error(e);
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
index 9271ebf..b8c8659 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
@@ -42,7 +42,7 @@
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
import edu.uci.ics.asterix.om.base.ABoolean;
import edu.uci.ics.asterix.om.base.ANull;
import edu.uci.ics.asterix.om.types.AOrderedListType;
@@ -375,7 +375,7 @@
private void parseYearMonthDuration(String duration, DataOutput out) throws AsterixException {
try {
- AYearMonthDurationerializerDeserializer.parse(duration, out);
+ AYearMonthDurationSerializerDeserializer.INSTANCE.parse(duration, out);
} catch (HyracksDataException e) {
throw new AsterixException(e);
}
@@ -383,7 +383,7 @@
private void parseDayTimeDuration(String duration, DataOutput out) throws AsterixException {
try {
- ADayTimeDurationSerializerDeserializer.parse(duration, out);
+ ADayTimeDurationSerializerDeserializer.INSTANCE.parse(duration, out);
} catch (HyracksDataException e) {
throw new AsterixException(e);
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
index 921dbf3..5a639dc 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
@@ -24,7 +24,9 @@
import edu.uci.ics.asterix.builders.IARecordBuilder;
import edu.uci.ics.asterix.builders.RecordBuilder;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ANullSerializerDeserializer;
import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.ANull;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -47,6 +49,8 @@
private int[] fldIds;
private ArrayBackedValueStorage[] nameBuffers;
+ private boolean areAllNullFields;
+
public DelimitedDataParser(ARecordType recordType, IValueParserFactory[] valueParserFactories, char fieldDelimter) {
this.recordType = recordType;
this.valueParserFactories = valueParserFactories;
@@ -97,29 +101,44 @@
@Override
public boolean parse(DataOutput out) throws AsterixException, IOException {
-
- if (cursor.nextRecord()) {
+ while (cursor.nextRecord()) {
recBuilder.reset(recordType);
recBuilder.init();
+ areAllNullFields = true;
for (int i = 0; i < valueParsers.length; ++i) {
if (!cursor.nextField()) {
break;
}
fieldValueBuffer.reset();
- fieldValueBufferOutput.writeByte(fieldTypeTags[i]);
- valueParsers[i]
- .parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart, fieldValueBufferOutput);
+
+ if (cursor.fStart == cursor.fEnd && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.STRING
+ && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.NULL) {
+ // if the field is empty and the type is optional, insert NULL
+ // note that string type can also process empty field as an empty string
+ if (recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.UNION) {
+ throw new AsterixException("Field " + i + " cannot be NULL. ");
+ }
+ fieldValueBufferOutput.writeByte(ATypeTag.NULL.serialize());
+ ANullSerializerDeserializer.INSTANCE.serialize(ANull.NULL, out);
+ } else {
+ fieldValueBufferOutput.writeByte(fieldTypeTags[i]);
+ valueParsers[i].parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart,
+ fieldValueBufferOutput);
+ areAllNullFields = false;
+ }
if (fldIds[i] < 0) {
recBuilder.addField(nameBuffers[i], fieldValueBuffer);
} else {
recBuilder.addField(fldIds[i], fieldValueBuffer);
}
+
}
- recBuilder.write(out, true);
- return true;
- } else {
- return false;
+ if (!areAllNullFields) {
+ recBuilder.write(out, true);
+ return true;
+ }
}
+ return false;
}
protected void fieldNameToBytes(String fieldName, AMutableString str, ArrayBackedValueStorage buffer)
@@ -252,11 +271,13 @@
if (p >= end) {
int s = start;
eof = !readMore();
+ p -= (s - start);
if (eof) {
state = State.EOF;
+ fStart = start;
+ fEnd = p;
return true;
}
- p -= (s - start);
}
char ch = buffer[p];
if (ch == fieldDelimiter) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
new file mode 100644
index 0000000..031a26e
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.opcallbacks;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
+
+public abstract class AbstractIndexModificationOperationCallback extends AbstractOperationCallback {
+
+ protected final long resourceId;
+ protected final byte resourceType;
+ protected final IndexOperation indexOp;
+ protected final ITransactionSubsystem txnSubsystem;
+ protected final SimpleTupleWriter tupleWriter;
+ protected final ILogRecord logRecord;
+
+ protected AbstractIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
+ ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
+ byte resourceType, IndexOperation indexOp) {
+ super(datasetId, primaryKeyFields, txnCtx, lockManager);
+ this.resourceId = resourceId;
+ this.resourceType = resourceType;
+ this.indexOp = indexOp;
+ this.txnSubsystem = txnSubsystem;
+ tupleWriter = new SimpleTupleWriter();
+ logRecord = new LogRecord();
+ logRecord.setTxnCtx(txnCtx);
+ logRecord.setLogType(LogType.UPDATE);
+ logRecord.setJobId(txnCtx.getJobId().getId());
+ logRecord.setDatasetId(datasetId);
+ logRecord.setResourceId(resourceId);
+ logRecord.setResourceType(resourceType);
+ logRecord.setNewOp((byte) (indexOp.ordinal()));
+ }
+
+ protected void log(int PKHash, ITupleReference newValue, IndexOperation oldOp, ITupleReference oldValue)
+ throws ACIDException {
+ logRecord.setPKHashValue(PKHash);
+ logRecord.setPKFields(primaryKeyFields);
+ logRecord.setPKValue(newValue);
+ logRecord.computeAndSetPKValueSize();
+ if (newValue != null) {
+ logRecord.setNewValueSize(tupleWriter.bytesRequired(newValue));
+ logRecord.setNewValue(newValue);
+ } else {
+ logRecord.setNewValueSize(0);
+ }
+ if (resourceType == ResourceType.LSM_BTREE) {
+ logRecord.setOldOp((byte) (oldOp.ordinal()));
+ if (oldValue != null) {
+ logRecord.setOldValueSize(tupleWriter.bytesRequired(oldValue));
+ logRecord.setOldValue(oldValue);
+ } else {
+ logRecord.setOldValueSize(0);
+ }
+ }
+ logRecord.computeAndSetLogSize();
+ txnSubsystem.getLogManager().log(logRecord);
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
index a6537cd..00f0c4a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
@@ -41,7 +41,7 @@
throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
try {
- ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
return new PrimaryIndexInstantSearchOperationCallback(datasetId, primaryKeyFields,
txnSubsystem.getLockManager(), txnCtx);
} catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
index 0024717..44dcad8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
@@ -16,12 +16,9 @@
package edu.uci.ics.asterix.transaction.management.opcallbacks;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -33,22 +30,13 @@
* Assumes LSM-BTrees as primary indexes.
* Performs locking on primary keys, and also logs before/after images.
*/
-public class PrimaryIndexModificationOperationCallback extends AbstractOperationCallback implements
+public class PrimaryIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback implements
IModificationOperationCallback {
- protected final long resourceId;
- protected final byte resourceType;
- protected final IndexOperation indexOp;
- protected final ITransactionSubsystem txnSubsystem;
-
public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType,
IndexOperation indexOp) {
- super(datasetId, primaryKeyFields, txnCtx, lockManager);
- this.resourceId = resourceId;
- this.resourceType = resourceType;
- this.indexOp = indexOp;
- this.txnSubsystem = txnSubsystem;
+ super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
}
@Override
@@ -64,7 +52,6 @@
@Override
public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
try {
- ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
IndexOperation oldOp = IndexOperation.INSERT;
@@ -74,8 +61,7 @@
if (lsmBTreeTuple != null && lsmBTreeTuple.isAntimatter()) {
oldOp = IndexOperation.DELETE;
}
- ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
- indexOp, after, oldOp, before);
+ log(pkHash, after, oldOp, before);
} catch (ACIDException e) {
throw new HyracksDataException(e);
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index 428a04e..07daee4 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -58,11 +58,11 @@
}
try {
- ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
indexOp);
- txnCtx.registerIndexAndCallback(index, (AbstractOperationCallback) modCallback);
+ txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
return modCallback;
} catch (ACIDException e) {
throw new HyracksDataException(e);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
index b59eb75..01cb725 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
@@ -41,7 +41,7 @@
throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
try {
- ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+ ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
return new PrimaryIndexSearchOperationCallback(datasetId, primaryKeyFields, txnSubsystem.getLockManager(),
txnCtx);
} catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
index e92f82c..ef5ad21 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
@@ -16,13 +16,10 @@
package edu.uci.ics.asterix.transaction.management.opcallbacks;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -34,24 +31,16 @@
* We assume that the modification of the corresponding primary index has already taken an appropriate lock.
* This callback performs logging of the before and/or after images for secondary indexes.
*/
-public class SecondaryIndexModificationOperationCallback extends AbstractOperationCallback implements
+public class SecondaryIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback implements
IModificationOperationCallback {
- protected final long resourceId;
- protected final byte resourceType;
- protected final IndexOperation indexOp;
protected final IndexOperation oldOp;
- protected final ITransactionSubsystem txnSubsystem;
public SecondaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
byte resourceType, IndexOperation indexOp) {
- super(datasetId, primaryKeyFields, txnCtx, lockManager);
- this.resourceId = resourceId;
- this.resourceType = resourceType;
- this.indexOp = indexOp;
+ super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
oldOp = (indexOp == IndexOperation.DELETE) ? IndexOperation.INSERT : IndexOperation.DELETE;
- this.txnSubsystem = txnSubsystem;
}
@Override
@@ -61,9 +50,7 @@
@Override
public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
-
try {
- ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
IndexOperation effectiveOldOp;
if (resourceType == ResourceType.LSM_BTREE) {
@@ -78,8 +65,7 @@
} else {
effectiveOldOp = oldOp;
}
- ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
- indexOp, after, effectiveOldOp, before);
+ this.log(pkHash, after, effectiveOldOp, before);
} catch (ACIDException e) {
throw new HyracksDataException(e);
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index 55ca399..563e9b7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -17,6 +17,7 @@
import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.asterix.common.transactions.AbstractOperationCallbackFactory;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
@@ -54,9 +55,11 @@
}
try {
- ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
- return new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
+ ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
+ IModificationOperationCallback modCallback = new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
+ txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
+ return modCallback;
} catch (ACIDException e) {
throw new HyracksDataException(e);
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
index 3661537..140a8dd 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
@@ -14,9 +14,10 @@
*/
package edu.uci.ics.asterix.transaction.management.opcallbacks;
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -24,15 +25,17 @@
private static final long serialVersionUID = 1L;
- private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+ private final int datasetID;
- public SecondaryIndexOperationTrackerProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
- this.ioOpCallbackFactory = ioOpCallbackFactory;
+ public SecondaryIndexOperationTrackerProvider(int datasetID) {
+ this.datasetID = datasetID;
}
@Override
public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
- return new BaseOperationTracker(ioOpCallbackFactory);
+ DatasetLifecycleManager dslcManager = (DatasetLifecycleManager) ((IAsterixAppRuntimeContext) ctx
+ .getJobletContext().getApplicationContext().getApplicationObject()).getIndexLifecycleManager();
+ return new BaseOperationTracker(dslcManager, datasetID);
}
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
index 8ce0174..cf60182 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
@@ -15,8 +15,10 @@
package edu.uci.ics.asterix.transaction.management.resource;
import java.io.File;
+import java.util.List;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -49,13 +51,15 @@
public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
int partition) {
FileReference file = new FileReference(new File(filePath));
- IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
- LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, runtimeContextProvider
+ List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getVirtualBufferCaches(datasetID);
+ LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, runtimeContextProvider
.getBufferCache(), runtimeContextProvider.getFileMapManager(), typeTraits, cmpFactories,
bloomFilterKeyFields, runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
- .getLSMMergePolicy(), isPrimary ? runtimeContextProvider.getLSMBTreeOperationTracker(datasetID)
- : new BaseOperationTracker(LSMBTreeIOOperationCallbackFactory.INSTANCE), runtimeContextProvider
- .getLSMIOScheduler(), runtimeContextProvider.getLSMBTreeIOOperationCallbackProvider(isPrimary));
+ .getLSMMergePolicy(),
+ isPrimary ? runtimeContextProvider.getLSMBTreeOperationTracker(datasetID) : new BaseOperationTracker(
+ (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(), datasetID),
+ runtimeContextProvider.getLSMIOScheduler(), LSMBTreeIOOperationCallbackFactory.INSTANCE
+ .createIOOperationCallback());
return lsmBTree;
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
index b3da3ee..5ed7019 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
@@ -14,7 +14,10 @@
*/
package edu.uci.ics.asterix.transaction.management.resource;
+import java.util.List;
+
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -53,26 +56,40 @@
@Override
public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
int partition) throws HyracksDataException {
- IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
+ List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getVirtualBufferCaches(datasetID);
try {
if (isPartitioned) {
- return InvertedIndexUtils.createPartitionedLSMInvertedIndex(virtualBufferCache, runtimeContextProvider
- .getFileMapManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(), filePath,
- runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
- .getLSMMergePolicy(), new BaseOperationTracker(
- LSMInvertedIndexIOOperationCallbackFactory.INSTANCE), runtimeContextProvider
- .getLSMIOScheduler(), runtimeContextProvider
- .getLSMInvertedIndexIOOperationCallbackProvider());
+ return InvertedIndexUtils.createPartitionedLSMInvertedIndex(
+ virtualBufferCaches,
+ runtimeContextProvider.getFileMapManager(),
+ invListTypeTraits,
+ invListCmpFactories,
+ tokenTypeTraits,
+ tokenCmpFactories,
+ tokenizerFactory,
+ runtimeContextProvider.getBufferCache(),
+ filePath,
+ runtimeContextProvider.getBloomFilterFalsePositiveRate(),
+ runtimeContextProvider.getLSMMergePolicy(),
+ new BaseOperationTracker((DatasetLifecycleManager) runtimeContextProvider
+ .getIndexLifecycleManager(), datasetID), runtimeContextProvider.getLSMIOScheduler(),
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE.createIOOperationCallback());
} else {
- return InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCache, runtimeContextProvider
- .getFileMapManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(), filePath,
- runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
- .getLSMMergePolicy(), new BaseOperationTracker(
- LSMInvertedIndexIOOperationCallbackFactory.INSTANCE), runtimeContextProvider
- .getLSMIOScheduler(), runtimeContextProvider
- .getLSMInvertedIndexIOOperationCallbackProvider());
+ return InvertedIndexUtils.createLSMInvertedIndex(
+ virtualBufferCaches,
+ runtimeContextProvider.getFileMapManager(),
+ invListTypeTraits,
+ invListCmpFactories,
+ tokenTypeTraits,
+ tokenCmpFactories,
+ tokenizerFactory,
+ runtimeContextProvider.getBufferCache(),
+ filePath,
+ runtimeContextProvider.getBloomFilterFalsePositiveRate(),
+ runtimeContextProvider.getLSMMergePolicy(),
+ new BaseOperationTracker((DatasetLifecycleManager) runtimeContextProvider
+ .getIndexLifecycleManager(), datasetID), runtimeContextProvider.getLSMIOScheduler(),
+ LSMInvertedIndexIOOperationCallbackFactory.INSTANCE.createIOOperationCallback());
}
} catch (IndexException e) {
throw new HyracksDataException(e);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
index 9b9faef..dc6b30b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
@@ -15,8 +15,10 @@
package edu.uci.ics.asterix.transaction.management.resource;
import java.io.File;
+import java.util.List;
import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -58,14 +60,15 @@
public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
int partition) throws HyracksDataException {
FileReference file = new FileReference(new File(filePath));
- IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
+ List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getVirtualBufferCaches(datasetID);
try {
- return LSMRTreeUtils.createLSMTree(virtualBufferCache, file, runtimeContextProvider.getBufferCache(),
+ return LSMRTreeUtils.createLSMTree(virtualBufferCaches, file, runtimeContextProvider.getBufferCache(),
runtimeContextProvider.getFileMapManager(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
valueProviderFactories, rtreePolicyType, runtimeContextProvider.getBloomFilterFalsePositiveRate(),
runtimeContextProvider.getLSMMergePolicy(), new BaseOperationTracker(
- LSMRTreeIOOperationCallbackFactory.INSTANCE), runtimeContextProvider.getLSMIOScheduler(),
- runtimeContextProvider.getLSMRTreeIOOperationCallbackProvider(), linearizeCmpFactory);
+ (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(), datasetID),
+ runtimeContextProvider.getLSMIOScheduler(), LSMRTreeIOOperationCallbackFactory.INSTANCE
+ .createIOOperationCallback(), linearizeCmpFactory);
} catch (TreeIndexException e) {
throw new HyracksDataException(e);
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
index b09cd52..f4b3a20 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
@@ -30,10 +30,13 @@
import edu.uci.ics.asterix.common.transactions.DatasetId;
import edu.uci.ics.asterix.common.transactions.ILockManager;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogPage;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogPageReader;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
@@ -78,38 +81,34 @@
private DeadlockDetector deadlockDetector;
private TimeOutDetector toutDetector;
private DatasetId tempDatasetIdObj; //temporary object to avoid object creation
+ private JobId tempJobIdObj;
private int tryLockDatasetGranuleRevertOperation;
private LockRequestTracker lockRequestTracker; //for debugging
private ConsecutiveWakeupContext consecutiveWakeupContext;
- //TODO
- //This code should be taken care properly when there is a way to avoid doubling memory space for txnIds.
- private LogicalLogLocator logicalLogLocator;
-
public LockManager(TransactionSubsystem txnSubsystem) throws ACIDException {
this.txnSubsystem = txnSubsystem;
this.lockTableLatch = new ReentrantReadWriteLock(true);
this.waiterLatch = new ReentrantReadWriteLock(true);
this.jobHT = new HashMap<JobId, JobInfo>();
this.datasetResourceHT = new HashMap<DatasetId, DatasetLockInfo>();
- this.entityInfoManager = new EntityInfoManager(txnSubsystem.getTransactionProperties().getLockManagerShrinkTimer());
+ this.entityInfoManager = new EntityInfoManager(txnSubsystem.getTransactionProperties()
+ .getLockManagerShrinkTimer());
this.lockWaiterManager = new LockWaiterManager();
this.entityLockInfoManager = new EntityLockInfoManager(entityInfoManager, lockWaiterManager);
this.deadlockDetector = new DeadlockDetector(jobHT, datasetResourceHT, entityLockInfoManager,
entityInfoManager, lockWaiterManager);
this.toutDetector = new TimeOutDetector(this);
this.tempDatasetIdObj = new DatasetId(0);
+ this.tempJobIdObj = new JobId(0);
this.consecutiveWakeupContext = new ConsecutiveWakeupContext();
-
- this.logicalLogLocator = LogUtil.getDummyLogicalLogLocator(txnSubsystem.getLogManager());
-
if (IS_DEBUG_MODE) {
this.lockRequestTracker = new LockRequestTracker();
}
}
-
+
public AsterixTransactionProperties getTransactionProperties() {
return this.txnSubsystem.getTransactionProperties();
}
@@ -200,7 +199,8 @@
if (doEscalate) {
throw new IllegalStateException(
"ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
- + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+ + txnSubsystem.getTransactionProperties()
+ .getEntityToDatasetLockEscalationThreshold());
}
}
}
@@ -304,9 +304,9 @@
}
private void validateJob(ITransactionContext txnContext) throws ACIDException {
- if (txnContext.getTxnState() == TransactionState.ABORTED) {
+ if (txnContext.getTxnState() == ITransactionManager.ABORTED) {
throw new ACIDException("" + txnContext.getJobId() + " is in ABORTED state.");
- } else if (txnContext.getStatus() == ITransactionContext.TIMED_OUT_STATUS) {
+ } else if (txnContext.isTimeout()) {
requestAbort(txnContext);
}
}
@@ -640,24 +640,17 @@
}
@Override
- public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
- throws ACIDException {
- return internalUnlock(datasetId, entityHashValue, txnContext, false, false);
- }
-
- @Override
- public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
- throws ACIDException {
- return internalUnlock(datasetId, entityHashValue, txnContext, false, commitFlag);
+ public void unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext) throws ACIDException {
+ internalUnlock(datasetId, entityHashValue, txnContext, false);
}
private void instantUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
throws ACIDException {
- internalUnlock(datasetId, entityHashValue, txnContext, true, false);
+ internalUnlock(datasetId, entityHashValue, txnContext, true);
}
- private boolean internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
- boolean isInstant, boolean commitFlag) throws ACIDException {
+ private void internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
+ boolean isInstant) throws ACIDException {
JobId jobId = txnContext.getJobId();
int eLockInfo = -1;
DatasetLockInfo dLockInfo = null;
@@ -665,7 +658,6 @@
int entityInfo = -1;
byte datasetLockMode;
- boolean lockCountIsZero = false;
if (IS_DEBUG_MODE) {
if (entityHashValue == -1) {
throw new UnsupportedOperationException(
@@ -675,8 +667,6 @@
latchLockTable();
try {
- validateJob(txnContext);
-
if (IS_DEBUG_MODE) {
trackLockRequest("Requested", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
dLockInfo, eLockInfo);
@@ -713,28 +703,11 @@
if (entityInfoManager.getEntityLockCount(entityInfo) == 0
&& entityInfoManager.getDatasetLockCount(entityInfo) == 0) {
- lockCountIsZero = true;
int threadCount = 0; //number of threads(in the same job) waiting on the same resource
int waiterObjId = jobInfo.getFirstWaitingResource();
int waitingEntityInfo;
LockWaiter waiterObj;
- //TODO
- //This code should be taken care properly when there is a way to avoid doubling memory space for txnIds.
- //This commit log is written here in order to avoid increasing the memory space for managing transactionIds
- if (commitFlag) {
- if (txnContext.getTransactionType().equals(ITransactionContext.TransactionType.READ_WRITE)) {
- try {
- txnSubsystem.getLogManager().log(LogType.ENTITY_COMMIT, txnContext, datasetId.getId(),
- entityHashValue, -1, (byte) 0, 0, null, null, logicalLogLocator);
- } catch (ACIDException e) {
- requestAbort(txnContext);
- }
- }
-
- txnContext.updateLastLSNForIndexes(logicalLogLocator.getLsn());
- }
-
//1) wake up waiters and remove holder
//wake up waiters of dataset-granule lock
wakeUpDatasetLockWaiters(dLockInfo);
@@ -760,11 +733,6 @@
waiterObjId = waiterObj.getNextWaiterObjId();
}
if (threadCount == 0) {
- if (entityInfoManager.getEntityLockMode(entityInfo) == LockMode.X) {
- //TODO
- //write a commit log for the unlocked resource
- //need to figure out that instantLock() also needs to write a commit log.
- }
entityInfoManager.deallocate(entityInfo);
}
}
@@ -782,7 +750,8 @@
if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
if (!isInstant && datasetLockMode == LockMode.IS) {
- jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+ jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties()
+ .getEntityToDatasetLockEscalationThreshold());
}
}
@@ -793,7 +762,6 @@
} finally {
unlatchLockTable();
}
- return lockCountIsZero;
}
@Override
@@ -974,8 +942,8 @@
jobHT.remove(jobId);
if (existWaiter) {
- txnContext.setStatus(ITransactionContext.TIMED_OUT_STATUS);
- txnContext.setTxnState(TransactionState.ABORTED);
+ txnContext.setTimeout(true);
+ txnContext.setTxnState(ITransactionManager.ABORTED);
}
if (IS_DEBUG_MODE) {
@@ -1297,7 +1265,8 @@
//We don't want to allow the lock escalation when there is a first lock request on a dataset.
throw new IllegalStateException(
"ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
- + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+ + txnSubsystem.getTransactionProperties()
+ .getEntityToDatasetLockEscalationThreshold());
}
}
}
@@ -1824,7 +1793,7 @@
//waiter woke up -> remove/deallocate waiter object and abort if timeout
latchLockTable();
- if (txnContext.getStatus() == ITransactionContext.TIMED_OUT_STATUS || waiter.isVictim()) {
+ if (txnContext.isTimeout() || waiter.isVictim()) {
requestAbort(txnContext);
}
@@ -1881,8 +1850,7 @@
}
private void requestAbort(ITransactionContext txnContext) throws ACIDException {
- txnContext.setStatus(ITransactionContext.TIMED_OUT_STATUS);
- txnContext.setStartWaitTime(ITransactionContext.INVALID_TIME);
+ txnContext.setTimeout(true);
throw new ACIDException("Transaction " + txnContext.getJobId()
+ " should abort (requested by the Lock Manager)");
}
@@ -2097,7 +2065,8 @@
try {
StringBuilder sb = new StringBuilder();
sb.append("\n>>dump_begin\t>>----- [ConfVars] -----");
- sb.append("\nESCALATE_TRHESHOLD_ENTITY_TO_DATASET: " + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+ sb.append("\nESCALATE_TRHESHOLD_ENTITY_TO_DATASET: "
+ + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
sb.append("\nSHRINK_TIMER_THRESHOLD (entityLockInfoManager): "
+ entityLockInfoManager.getShrinkTimerThreshold());
sb.append("\nSHRINK_TIMER_THRESHOLD (entityInfoManager): " + entityInfoManager.getShrinkTimerThreshold());
@@ -2233,6 +2202,31 @@
}
}
}
+
+ public void batchUnlock(LogPage logPage, LogPageReader logPageReader) throws ACIDException {
+ latchLockTable();
+ try {
+ ITransactionContext txnCtx = null;
+ LogRecord logRecord = logPageReader.next();
+ while (logRecord != null) {
+ if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
+ tempDatasetIdObj.setId(logRecord.getDatasetId());
+ tempJobIdObj.setId(logRecord.getJobId());
+ txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
+ unlock(tempDatasetIdObj, logRecord.getPKHashValue(), txnCtx);
+ txnCtx.notifyOptracker(false);
+ } else if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+ tempJobIdObj.setId(logRecord.getJobId());
+ txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
+ txnCtx.notifyOptracker(true);
+ ((LogPage) logPage).notifyJobTerminator();
+ }
+ logRecord = logPageReader.next();
+ }
+ } finally {
+ unlatchLockTable();
+ }
+ }
}
class ConsecutiveWakeupContext {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
index e97c177..e61cb55 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.asterix.transaction.management.service.locking;
+import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.ArrayList;
@@ -21,17 +22,15 @@
import java.util.NoSuchElementException;
import java.util.Scanner;
-import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
-import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
-import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import org.apache.commons.io.FileUtils;
+
import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.common.transactions.DatasetId;
import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
@@ -40,6 +39,13 @@
public class LockManagerDeterministicUnitTest {
public static void main(String args[]) throws ACIDException, IOException, AsterixException {
+ //prepare configuration file
+ File cwd = new File(System.getProperty("user.dir"));
+ File asterixdbDir = cwd.getParentFile();
+ File srcFile = new File(asterixdbDir.getAbsoluteFile(), "asterix-app/src/main/resources/asterix-build-configuration.xml");
+ File destFile = new File(cwd, "target/classes/asterix-configuration.xml");
+ FileUtils.copyFile(srcFile, destFile);
+
//initialize controller thread
String requestFileName = new String(
"src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile");
@@ -61,8 +67,8 @@
long defaultWaitTime;
public LockRequestController(String requestFileName) throws ACIDException, AsterixException {
- this.txnProvider = new TransactionSubsystem("LockManagerPredefinedUnitTest", null,
- new AsterixTransactionProperties(new AsterixPropertiesAccessor()));
+ this.txnProvider = new TransactionSubsystem("nc1", null, new AsterixTransactionProperties(
+ new AsterixPropertiesAccessor()));
this.workerReadyQueue = new WorkerReadyQueue();
this.requestList = new ArrayList<LockRequest>();
this.expectedResultList = new ArrayList<ArrayList<Integer>>();
@@ -428,9 +434,9 @@
try {
sendRequest(lockRequest);
} catch (ACIDException e) {
- if (lockRequest.txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
- if (lockRequest.txnContext.getTxnState() != TransactionState.ABORTED) {
- lockRequest.txnContext.setTxnState(TransactionState.ABORTED);
+ if (lockRequest.txnContext.isTimeout()) {
+ if (lockRequest.txnContext.getTxnState() != ITransactionManager.ABORTED) {
+ lockRequest.txnContext.setTxnState(ITransactionManager.ABORTED);
log("*** " + lockRequest.txnContext.getJobId() + " lock request causing deadlock ***");
log("Abort --> Releasing all locks acquired by " + lockRequest.txnContext.getJobId());
try {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
index f73460d..e6f2798 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
@@ -24,7 +24,7 @@
import edu.uci.ics.asterix.common.transactions.DatasetId;
import edu.uci.ics.asterix.common.transactions.ILockManager;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
@@ -287,8 +287,8 @@
} else {
try {
synchronized (txnContext) {
- if (txnContext.getTxnState() != TransactionState.ABORTED) {
- txnContext.setTxnState(TransactionState.ABORTED);
+ if (txnContext.getTxnState() != ITransactionManager.ABORTED) {
+ txnContext.setTxnState(ITransactionManager.ABORTED);
mayRelease = true;
}
}
@@ -446,9 +446,9 @@
try {
lockMgr.lock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
} catch (ACIDException e) {
- if (request.txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
- if (request.txnContext.getTxnState() != TransactionState.ABORTED) {
- request.txnContext.setTxnState(TransactionState.ABORTED);
+ if (request.txnContext.isTimeout()) {
+ if (request.txnContext.getTxnState() != ITransactionManager.ABORTED) {
+ request.txnContext.setTxnState(ITransactionManager.ABORTED);
log("*** " + request.txnContext.getJobId() + " lock request causing deadlock ***");
log("Abort --> Releasing all locks acquired by " + request.txnContext.getJobId());
try {
@@ -469,9 +469,9 @@
lockMgr.instantLock(request.datasetIdObj, request.entityHashValue, request.lockMode,
request.txnContext);
} catch (ACIDException e) {
- if (request.txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
- if (request.txnContext.getTxnState() != TransactionState.ABORTED) {
- request.txnContext.setTxnState(TransactionState.ABORTED);
+ if (request.txnContext.isTimeout()) {
+ if (request.txnContext.getTxnState() != ITransactionManager.ABORTED) {
+ request.txnContext.setTxnState(ITransactionManager.ABORTED);
log("*** " + request.txnContext.getJobId() + " lock request causing deadlock ***");
log("Abort --> Releasing all locks acquired by " + request.txnContext.getJobId());
try {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile
index fc2a883..c755895 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile
@@ -1,20 +1,11 @@
T1 L J1 D1 E1 S
-T3 L J3 D1 E-1 S
-T2 L J2 D1 E-1 X
-T4 L J4 D1 E1 S
-T0 CST 1 3 -1
-T1 L J1 D1 E2 X
-T0 CST 3 -1
-T3 RL J3 D1 E-1 S
-T0 CST 1 3 -1
+T3 L J3 D1 E1 S
+T2 L J2 D1 E1 X
+T0 CSQ 1 3 -1
T1 UL J1 D1 E1 S
-T0 CST 1 3 -1
-T1 UL J1 D1 E2 X
-T0 CST 1 2 3 -1
-T3 END
+T3 UL J3 D1 E1 S
+T0 CST 1 2 3 -1
+T2 RL J2 D1 E1 X
T1 END
-T2 RL J2 D1 E-1 X
T2 END
-T0 CST 4 -1
-T4 UL J4 D1 E1 S
-T4 END
\ No newline at end of file
+T3 END
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
deleted file mode 100644
index 272ef5e..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support. This class is not "thread-safe". For performance
- * concerns, it is required for multiple writers to be able to write to the
- * buffer concurrently and that a writer is never blocked by another writer. The
- * users of this class must ensure that two concurrent writers get to write in
- * exclusive areas in the buffer. A reader and writer may or may not conflict
- * with each other. For example, reading of logs during roll back of a
- * transaction t1 does not conflict with writing of logs by another transaction
- * t2 as they are concerned with exclusive areas of the buffer. On the contrary,
- * a flushing the buffer to disk conflicts with a reader reading the buffer.
- * Appropriate locks are taken on the Buffer in the application logic and not
- * directly imposed by synchronized methods.
- */
-
-public class Buffer implements IBuffer {
-
- ByteBuffer buffer;
-
- public Buffer(ByteBuffer buffer) {
- this.buffer = buffer;
- }
-
- protected Buffer() {
- }
-
- public void setBuffer(ByteBuffer buffer) {
- this.buffer = buffer;
- }
-
- @Override
- public byte getByte(int offset) {
- return buffer.get(offset);
- }
-
- @Override
- public byte getByte() {
- return buffer.get();
- }
-
- @Override
- public void getBytes(byte[] bytes, int offset, int size) {
- System.arraycopy(buffer.array(), offset, bytes, 0, size);
- }
-
- @Override
- public int getSize() {
- return buffer.capacity();
- }
-
- @Override
- public int readInt() {
- return buffer.getInt();
- }
-
- @Override
- public int readInt(int offset) {
- return buffer.getInt(offset);
- }
-
- @Override
- public long readLong(int offset) {
- return buffer.getLong(offset);
- }
-
- @Override
- public void put(byte b) {
- buffer.put(b);
- }
-
- @Override
- public void put(int offset, byte b) {
- buffer.put(offset, b);
- }
-
- @Override
- public void put(byte[] bytes, int start, int length) {
- buffer.put(bytes, start, length);
-
- }
-
- @Override
- public void put(byte[] bytes) {
- buffer.put(bytes);
- }
-
- @Override
- public void writeInt(int value) {
- buffer.putInt(value);
- }
-
- @Override
- public void writeInt(int offset, int value) {
- buffer.putInt(offset, value);
-
- }
-
- @Override
- public void writeLong(long value) {
- buffer.putLong(value);
- }
-
- @Override
- public void writeLong(int offset, long value) {
- buffer.putLong(offset, value);
-
- }
-
- @Override
- public byte[] getArray() {
- return buffer.array();
- }
-
- @Override
- public void erase() {
- Arrays.fill(buffer.array(), (byte) 0);
- }
-
- @Override
- public ByteBuffer getByteBuffer() {
- return buffer;
- }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
deleted file mode 100644
index 79c95c4..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.util.zip.CRC32;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * A utility class for doing bit level operations such as forming checksum or
- * converting between Integer and byte array. Used extensively during writing
- * and reading of logs.
- */
-public class DataUtil {
-
- public static long getChecksum(IBuffer buffer, int offset, int length) {
- CRC32 checksumEngine = new CRC32();
- byte[] bytes = new byte[1];
- for (int i = 0; i < length; i++) {
- bytes[0] = buffer.getByte(offset + i);
- checksumEngine.update(bytes, 0, 1);
- }
- return checksumEngine.getValue();
- }
-
- public static int byteArrayToInt(byte[] bytes, int offset) {
- return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
- + ((bytes[offset + 3] & 0xff) << 0);
- }
-
- public static byte[] intToByteArray(int value) {
- byte[] bytes = new byte[4];
- bytes[0] = (byte) ((value >>> 24) & 0xFF);
- bytes[1] = (byte) ((value >>> 16) & 0xFF);
- bytes[2] = (byte) ((value >>> 8) & 0xFF);
- bytes[3] = (byte) ((value >>> 0) & 0xFF);
- return bytes;
- }
-
- public static long byteArrayToLong(byte[] bytes, int offset) {
- return ((bytes[offset] & 0xff) << 56) + ((bytes[offset + 1] & 0xff) << 48) + ((bytes[offset + 2] & 0xff) << 40)
- + ((bytes[offset + 3] & 0xff) << 32) + ((bytes[offset + 4] & 0xff) << 24)
- + ((bytes[offset + 5] & 0xff) << 16) + ((bytes[offset + 6] & 0xff) << 8)
- + ((bytes[offset + 7] & 0xff) << 0);
- }
-
- public static byte[] longToByteArray(long value) {
- byte[] bytes = new byte[8];
- bytes[0] = (byte) ((value >>> 56) & 0xFF);
- bytes[1] = (byte) ((value >>> 48) & 0xFF);
- bytes[2] = (byte) ((value >>> 40) & 0xFF);
- bytes[3] = (byte) ((value >>> 32) & 0xFF);
- bytes[4] = (byte) ((value >>> 24) & 0xFF);
- bytes[5] = (byte) ((value >>> 16) & 0xFF);
- bytes[6] = (byte) ((value >>> 8) & 0xFF);
- bytes[7] = (byte) ((value >>> 0) & 0xFF);
- return bytes;
- }
-
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
deleted file mode 100644
index 0de43cc..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-
-/*
- * A generic filter that encompasses one or more filters (see @ILogFiler) that may be applied when selectively retrieving logs.
- * The contained filters are assumed to form a conjunction.
- */
-public class GenericFilter implements ILogFilter {
-
- private final List<ILogFilter> logFilters;
-
- public GenericFilter() {
- logFilters = new ArrayList<ILogFilter>();
- }
-
- public GenericFilter(List<ILogFilter> logFilters) {
- this.logFilters = logFilters;
- }
-
- public boolean accept(IBuffer fileBuffer, long offset, int length) {
- boolean satisfies = true;
- for (ILogFilter logFilter : logFilters) {
- satisfies = satisfies && logFilter.accept(fileBuffer, offset, length);
- if (!satisfies) {
- break;
- }
- }
- return satisfies;
- }
-
- public void addFilter(ILogFilter logFilter) {
- logFilters.add(logFilter);
- }
-
- public boolean removeFilter(ILogFilter logFilter) {
- return logFilters.remove(logFilter);
- }
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
deleted file mode 100644
index b95b943..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ILoggerRepository;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.MutableResourceId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
-
-public class IndexLoggerRepository implements ILoggerRepository {
-
- private final Map<MutableResourceId, ILogger> loggers = new HashMap<MutableResourceId, ILogger>();
- private final ITransactionSubsystem txnSubsystem;
- private MutableResourceId mutableResourceId;
-
- public IndexLoggerRepository(ITransactionSubsystem provider) {
- this.txnSubsystem = provider;
- mutableResourceId = new MutableResourceId(0);
- }
-
- @Override
- public synchronized ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException {
- mutableResourceId.setId(resourceId);
- ILogger logger = loggers.get(mutableResourceId);
- if (logger == null) {
- MutableResourceId newMutableResourceId = new MutableResourceId(resourceId);
- IIndex index;
- try {
- index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
- .getIndex(resourceId);
- } catch (HyracksDataException e) {
- throw new ACIDException(e);
- }
- logger = new IndexLogger(resourceId, resourceType, index);
- loggers.put(newMutableResourceId, logger);
- }
- return logger;
- }
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
deleted file mode 100644
index 16ffa69..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-
-public class IndexResourceManager implements IResourceManager {
-
- public final byte resourceType;
-
- private final ITransactionSubsystem txnSubsystem;
-
- public IndexResourceManager(byte resourceType, ITransactionSubsystem provider) {
- this.resourceType = resourceType;
- this.txnSubsystem = provider;
- }
-
- public byte getResourceManagerId() {
- return resourceType;
- }
-
- public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
- long resourceId = logRecordHelper.getResourceId(logLocator);
- int offset = logRecordHelper.getLogContentBeginPos(logLocator);
-
- IIndex index;
- try {
- index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
- .getIndex(resourceId);
- } catch (HyracksDataException e1) {
- throw new ACIDException("Cannot undo: unable to find index");
- }
-
- /* field count */
- int fieldCount = logLocator.getBuffer().readInt(offset);
- offset += 4;
-
- /* new operation */
- byte newOperation = logLocator.getBuffer().getByte(offset);
- offset += 1;
-
- /* new value size */
- int newValueSize = logLocator.getBuffer().readInt(offset);
- offset += 4;
-
- /* new value */
- SimpleTupleWriter tupleWriter = new SimpleTupleWriter();
- SimpleTupleReference newTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
- newTuple.setFieldCount(fieldCount);
- newTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
- offset += newValueSize;
-
- try {
- ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-
- if (resourceType == ResourceType.LSM_BTREE) {
-
- /* old operation */
- byte oldOperation = logLocator.getBuffer().getByte(offset);
- offset += 1;
-
- if (oldOperation != (byte) IndexOperation.NOOP.ordinal()) {
- /* old value size */
- int oldValueSize = logLocator.getBuffer().readInt(offset);
- offset += 4;
-
- /* old value */
- SimpleTupleReference oldTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
- oldTuple.setFieldCount(fieldCount);
- oldTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
- offset += oldValueSize;
-
- if (oldOperation == (byte) IndexOperation.DELETE.ordinal()) {
- indexAccessor.forceDelete(oldTuple);
- } else {
- indexAccessor.forceInsert(oldTuple);
- }
- } else {
- indexAccessor.forcePhysicalDelete(newTuple);
- }
- } else {
- if (newOperation == (byte) IndexOperation.DELETE.ordinal()) {
- indexAccessor.forceInsert(newTuple);
- } else {
- indexAccessor.forceDelete(newTuple);
- }
- }
- } catch (Exception e) {
- throw new ACIDException("Undo failed", e);
- }
- }
-
- public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
- long resourceId = logRecordHelper.getResourceId(logLocator);
- int offset = logRecordHelper.getLogContentBeginPos(logLocator);
-
- IIndex index;
- try {
- index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
- .getIndex(resourceId);
- } catch (HyracksDataException e1) {
- throw new ACIDException("Cannot redo: unable to find index");
- }
-
- /* field count */
- int fieldCount = logLocator.getBuffer().readInt(offset);
- offset += 4;
-
- /* new operation */
- byte newOperation = logLocator.getBuffer().getByte(offset);
- offset += 1;
-
- /* new value size */
- int newValueSize = logLocator.getBuffer().readInt(offset);
- offset += 4;
-
- /* new value */
- SimpleTupleWriter tupleWriter = new SimpleTupleWriter();
- SimpleTupleReference newTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
- newTuple.setFieldCount(fieldCount);
- newTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
- offset += newValueSize;
-
- try {
- ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-
- if (newOperation == IndexOperation.INSERT.ordinal()) {
- indexAccessor.insert(newTuple);
- } else if (newOperation == IndexOperation.DELETE.ordinal()) {
- indexAccessor.delete(newTuple);
- } else {
- new ACIDException("Unsupported operation type for undo operation : " + newOperation);
- }
- } catch (Exception e) {
- throw new ACIDException("Redo failed", e);
- }
- }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java
deleted file mode 100644
index 5f558f3..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-public class LogActionType {
-
- public static final byte REDO = 0; // used for a log record that contains
- // just redo information.
- public static final byte REDO_UNDO = 1; // used for a log record that
- // contains both redo and undo
- // information.
- public static final byte UNDO = 2; // used for a log record that contains
- // just undo information.
- public static final byte NO_OP = 3; // used for a log record that does not
- // require any action.
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index b8d5312..9551ba8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -15,746 +15,197 @@
package edu.uci.ics.asterix.transaction.management.service.logging;
import java.io.File;
+import java.io.FilenameFilter;
import java.io.IOException;
import java.io.OutputStream;
import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Collections;
+import java.util.Comparator;
import java.util.List;
+<<<<<<< HEAD
import java.util.Map;
import java.util.Set;
+=======
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+>>>>>>> master
import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.logging.Level;
import java.util.logging.Logger;
import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.FileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.IFileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
+<<<<<<< HEAD
import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
+=======
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
+>>>>>>> master
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
public class LogManager implements ILogManager, ILifeCycleComponent {
- public static final boolean IS_DEBUG_MODE = false;//true
+ public static final boolean IS_DEBUG_MODE = false;// true
private static final Logger LOGGER = Logger.getLogger(LogManager.class.getName());
- private final TransactionSubsystem provider;
- private LogManagerProperties logManagerProperties;
- private LogPageFlushThread logPageFlusher;
+ private final TransactionSubsystem txnSubsystem;
+ private final LogManagerProperties logManagerProperties;
+ private final long logFileSize;
private final int logPageSize;
- private long statLogSize;
- private long statLogCount;
+ private final int numLogPages;
+ private final String logDir;
+ private final String logFilePrefix;
+ private final MutableLong flushLSN;
+ private LinkedBlockingQueue<LogPage> emptyQ;
+ private LinkedBlockingQueue<LogPage> flushQ;
+ private long appendLSN;
+ private FileChannel appendChannel;
+ private LogPage appendPage;
+ private LogFlusher logFlusher;
+ private Future<Object> futureLogFlusher;
- /*
- * the array of log pages. The number of log pages is configurable. Pages
- * taken together form an in-memory log buffer.
- */
- private IFileBasedBuffer[] logPages;
-
- private ILogRecordHelper logRecordHelper;
-
- /*
- * Number of log pages that constitute the in-memory log buffer.
- */
- private int numLogPages;
-
- private AtomicLong lastFlushedLSN = new AtomicLong(-1);
-
- /*
- * When the transaction eco-system comes to life, the log manager positions
- * itself to the end of the last written log. the startingLsn represent the
- * lsn value of the next log record to be written after a system (re)start.
- * The value is zero when the system is starting for the first time.
- */
- private long startingLSN = 0;
-
- /*
- * lsn represents the monotonically increasing long value that can be broken
- * down into a file id and an offset within a log file.
- */
- private AtomicLong lsn = new AtomicLong(0);
-
- private List<HashMap<ITransactionContext, Integer>> activeTxnCountMaps;
-
- public void addFlushRequest(int pageIndex, long lsn, boolean isSynchronous) {
- logPageFlusher.requestFlush(pageIndex, lsn, isSynchronous);
- }
-
- public AtomicLong getLastFlushedLsn() {
- return lastFlushedLSN;
- }
-
- public AtomicLong getCurrentLsn() {
- return lsn;
- }
-
- public long incrementLastFlushedLsn(long delta) {
- return lastFlushedLSN.addAndGet(delta);
- }
-
- public LogManager(TransactionSubsystem provider) throws ACIDException {
- this.provider = provider;
- logManagerProperties = new LogManagerProperties(this.provider.getTransactionProperties(), this.provider.getId());
+ public LogManager(TransactionSubsystem txnSubsystem) throws ACIDException {
+ this.txnSubsystem = txnSubsystem;
+ logManagerProperties = new LogManagerProperties(this.txnSubsystem.getTransactionProperties(),
+ this.txnSubsystem.getId());
+ logFileSize = logManagerProperties.getLogPartitionSize();
logPageSize = logManagerProperties.getLogPageSize();
- initLogManager();
- statLogSize = 0;
- statLogCount = 0;
- }
-
- public LogManager(TransactionSubsystem provider, String nodeId) throws ACIDException {
- this.provider = provider;
- logManagerProperties = new LogManagerProperties(provider.getTransactionProperties(), nodeId);
- logPageSize = logManagerProperties.getLogPageSize();
- initLogManager();
- statLogSize = 0;
- statLogCount = 0;
- }
-
- private void initLogManager() throws ACIDException {
- logRecordHelper = new LogRecordHelper(this);
numLogPages = logManagerProperties.getNumLogPages();
- activeTxnCountMaps = new ArrayList<HashMap<ITransactionContext, Integer>>(numLogPages);
+ logDir = logManagerProperties.getLogDir();
+ logFilePrefix = logManagerProperties.getLogFilePrefix();
+ flushLSN = new MutableLong();
+ initializeLogManager(0);
+ }
+ private void initializeLogManager(long nextLogFileId) {
+ emptyQ = new LinkedBlockingQueue<LogPage>(numLogPages);
+ flushQ = new LinkedBlockingQueue<LogPage>(numLogPages);
for (int i = 0; i < numLogPages; i++) {
- activeTxnCountMaps.add(new HashMap<ITransactionContext, Integer>());
+ emptyQ.offer(new LogPage((LockManager) txnSubsystem.getLockManager(), logPageSize, flushLSN));
}
-
- logPages = new FileBasedBuffer[numLogPages];
-
- /*
- * place the log anchor at the end of the last log record written.
- */
- initLSN();
-
- /*
- * initialize the log pages.
- */
- initializeLogPages(startingLSN);
-
- /*
- * Instantiate and begin the LogFlusher thread. The Log Flusher thread
- * is responsible for putting log pages to disk. It is configured as a
- * daemon thread so that it does not stop the JVM from exiting when all
- * other threads are done with their work.
- */
- logPageFlusher = new LogPageFlushThread(this);
- logPageFlusher.setDaemon(true);
- AsterixThreadExecutor.INSTANCE.execute(logPageFlusher);
- }
-
- public int getLogPageIndex(long lsnValue) {
- return (int) (((lsnValue - startingLSN) / logPageSize) % numLogPages);
- }
-
- /*
- * given a lsn, get the file id where the corresponding log record is
- * located.
- */
- public int getLogFileId(long lsnValue) {
- return (int) ((lsnValue) / logManagerProperties.getLogPartitionSize());
- }
-
- /*
- * given a lsn, get the offset within a log page where the corresponding log
- * record is (to be) placed.
- */
- public int getLogPageOffset(long lsnValue) {
- return (int) (lsnValue % logPageSize);
- }
-
- /*
- * The method that reserves the space for a transaction to write log record
- * in the log buffer. Note that the method is not synchronized for
- * performance reasons as we do not want transactions to be blocked by each
- * other when writing log records.
- *
- * @param entrySize: the requested space.
- *
- * @param logType: the type of log record.
- */
- private long getLsn(int entrySize, byte logType) throws ACIDException {
-
- while (true) {
- boolean forwardPage = false;
- long old = lsn.get();
-
- // get the log page corresponding to the current lsn value
- int pageIndex = getLogPageIndex(old);
- long retVal = old;
-
- // the lsn value for the next request if the current request is
- // served.
- long next = old + entrySize;
- int prevPage = -1;
-
- // check if the log record will cross page boundaries, a case that
- // is not allowed.
- if ((next - 1) / logPageSize != old / logPageSize || (next % logPageSize == 0)) {
-
- if ((old != 0 && old % logPageSize == 0)) {
- // On second thought, this shall never be the case as it
- // means that the lsn is
- // currently at the beginning of a page and we still need to
- // forward the page which
- // means that the entrySize exceeds a log page size. If this
- // is the case, an
- // exception is thrown before calling this API. would remove
- // this case.
- retVal = old;
-
- } else {
- // set the lsn to point to the beginning of the next page.
- retVal = ((old / logPageSize) + 1) * logPageSize;
- }
-
- next = retVal;
-
- // as the log record shall cross log page boundary, we must
- // re-assign the lsn so
- // that the log record begins on a different location.
- forwardPage = true;
-
- prevPage = pageIndex;
- pageIndex = getNextPageInSequence(pageIndex);
- }
-
- if (!lsn.compareAndSet(old, next)) {
- // Atomic call -> returns true only when the value represented
- // by lsn is same as
- // "old". The value is updated to "next".
- continue;
- }
-
- if (forwardPage) {
- logPages[prevPage].acquireReadLatch();
- // increment the counter as the transaction thread now holds a
- // space in the log page and hence is an owner.
- logPages[prevPage].incRefCnt();
- logPages[prevPage].releaseReadLatch();
-
- // forward the nextWriteOffset in the log page
- logPages[prevPage].setBufferNextWriteOffset(logPageSize);
-
- logPages[prevPage].decRefCnt();
-
- addFlushRequest(prevPage, old, false);
-
- // The transaction thread that discovers the need to forward a
- // page is made to re-acquire a lsn.
- continue;
-
- } else {
- logPages[pageIndex].acquireReadLatch();
- // increment the counter as the transaction thread now holds a
- // space in the log page and hence is an owner.
- logPages[pageIndex].incRefCnt();
- logPages[pageIndex].releaseReadLatch();
-
- // Before the count is incremented, if the flusher flushed the
- // allocated page,
- // then retry to get new LSN. Otherwise, the log with allocated
- // lsn will be lost.
- if (lastFlushedLSN.get() >= retVal) {
- logPages[pageIndex].decRefCnt();
- continue;
- }
- }
-
- return retVal;
- }
- }
-
- @Override
- public void log(byte logType, ITransactionContext txnCtx, int datasetId, int PKHashValue, long resourceId,
- byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
- LogicalLogLocator logicalLogLocator) throws ACIDException {
-
- HashMap<ITransactionContext, Integer> map = null;
- int activeTxnCount;
-
- // logLocator is a re-usable object that is appropriately set in each
- // invocation.
- // If the reference is null, the log manager must throw an exception.
- if (logicalLogLocator == null) {
- throw new ACIDException(
- " you need to pass in a non-null logLocator, if you dont have it, then pass in a dummy so that the +"
- + "log manager can set it approporiately for you");
- }
-
- // compute the total log size including the header and the checksum.
- int totalLogSize = logRecordHelper.getLogRecordSize(logType, logContentSize);
-
- // check for the total space requirement to be less than a log page.
- if (totalLogSize > logPageSize) {
- throw new ACIDException(
- " Maximum Log Content Size is "
- + (logPageSize - logRecordHelper.getLogHeaderSize(LogType.UPDATE) - logRecordHelper
- .getLogChecksumSize()));
- }
-
- // all constraints checked and we are good to go and acquire a lsn.
- long previousLSN = -1;
-
- // the will be set to the location (a long value) where the log record
- // needs to be placed.
- long currentLSN;
-
- // The logs written by a transaction need to be linked to each other for
- // a successful rollback/recovery. However there could be multiple
- // threads operating concurrently that are part of a common transaction.
- // These threads need to synchronize and record the lsn corresponding to
- // the last log record written by (any thread of) the transaction.
- synchronized (txnCtx) {
- previousLSN = txnCtx.getLastLogLocator().getLsn();
- currentLSN = getLsn(totalLogSize, logType);
- txnCtx.setLastLSN(currentLSN);
- if (IS_DEBUG_MODE) {
- System.out.println("--------------> LSN(" + currentLSN + ") is allocated");
- }
- logicalLogLocator.setLsn(currentLSN);
- }
-
- /*
- * At this point, the transaction thread has obtained reserved space for
- * writing the log record. In doing so, it has acquired (shared)
- * ownership of the log page. All subsequent actions are under a try
- * catch block so that if any exception is encountered, a clean can be
- * performed correctly that is ownership is released.
- */
-
- // indicates if the transaction thread has release ownership of the
- // page.
- boolean decremented = false;
-
- int pageIndex = (int) getLogPageIndex(currentLSN);
-
- // the lsn has been obtained for the log record. need to set the
- // LogLocator instance accordingly.
- try {
- logicalLogLocator.setBuffer(logPages[pageIndex]);
- int pageOffset = getLogPageOffset(currentLSN);
- logicalLogLocator.setMemoryOffset(pageOffset);
-
- // write the log header.
- logRecordHelper.writeLogHeader(logicalLogLocator, logType, txnCtx, datasetId, PKHashValue, previousLSN,
- resourceId, resourceMgrId, logContentSize);
-
- // increment the offset so that the transaction can fill up the
- // content in the correct region of the allocated space.
- logicalLogLocator.increaseMemoryOffset(logRecordHelper.getLogHeaderSize(logType));
-
- // a COMMIT log record does not have any content and hence
- // the logger (responsible for putting the log content) is not
- // invoked.
- if (logContentSize != 0) {
- logger.preLog(txnCtx, reusableLogContentObject);
- }
-
- if (logContentSize != 0) {
- // call the logger implementation and ask to fill in the log
- // record content at the allocated space.
- logger.log(txnCtx, logicalLogLocator, logContentSize, reusableLogContentObject);
- logger.postLog(txnCtx, reusableLogContentObject);
- if (IS_DEBUG_MODE) {
- logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset()
- - logRecordHelper.getLogHeaderSize(logType));
- System.out.println(logRecordHelper.getLogRecordForDisplay(logicalLogLocator));
- logicalLogLocator.increaseMemoryOffset(logRecordHelper.getLogHeaderSize(logType));
- }
- }
-
- // The log record has been written. For integrity checks, compute
- // the checksum and put it at the end of the log record.
- int startPosChecksum = logicalLogLocator.getMemoryOffset() - logRecordHelper.getLogHeaderSize(logType);
- int length = totalLogSize - logRecordHelper.getLogChecksumSize();
- long checksum = DataUtil.getChecksum(logPages[pageIndex], startPosChecksum, length);
- logPages[pageIndex].writeLong(pageOffset + logRecordHelper.getLogHeaderSize(logType) + logContentSize,
- checksum);
-
- // forward the nextWriteOffset in the log page
- int bufferNextWriteOffset = (int) ((currentLSN + totalLogSize) % logPageSize);
- if (bufferNextWriteOffset == 0) {
- bufferNextWriteOffset = logPageSize;
- }
- logPages[pageIndex].setBufferNextWriteOffset(bufferNextWriteOffset);
-
- if (logType != LogType.ENTITY_COMMIT) {
- if (logType == LogType.COMMIT) {
- txnCtx.setExclusiveJobLevelCommit();
- map = activeTxnCountMaps.get(pageIndex);
- map.put(txnCtx, 1);
- }
- // release the ownership as the log record has been placed in
- // created space.
- logPages[pageIndex].decRefCnt();
-
- // indicating that the transaction thread has released ownership
- decremented = true;
- }
-
- if (logType == LogType.ENTITY_COMMIT) {
- map = activeTxnCountMaps.get(pageIndex);
- if (map.containsKey(txnCtx)) {
- activeTxnCount = (Integer) map.get(txnCtx);
- activeTxnCount++;
- map.put(txnCtx, activeTxnCount);
- } else {
- map.put(txnCtx, 1);
- }
- //------------------------------------------------------------------------------
- // [Notice]
- // reference count should be decremented
- // after activeTxnCount is incremented, but before addFlushRequest() is called.
- //------------------------------------------------------------------------------
- // release the ownership as the log record has been placed in
- // created space.
- logPages[pageIndex].decRefCnt();
-
- // indicating that the transaction thread has released ownership
- decremented = true;
-
- addFlushRequest(pageIndex, currentLSN, false);
- } else if (logType == LogType.COMMIT) {
-
- addFlushRequest(pageIndex, currentLSN, true);
- if (IS_DEBUG_MODE) {
- System.out.println("Running sum of log size: " + statLogSize + ", log count: " + statLogCount);
- }
- }
-
- if (IS_DEBUG_MODE) {
- System.out.println("--------------> LSN(" + currentLSN + ") is written");
- }
-
- //collect statistics
- statLogSize += totalLogSize;
- statLogCount++;
-
- } catch (Exception e) {
- e.printStackTrace();
- throw new ACIDException(txnCtx, "Thread: " + Thread.currentThread().getName()
- + " logger encountered exception", e);
- } finally {
- if (!decremented) {
- logPages[pageIndex].decRefCnt();
- }
- }
- }
-
- /*
- * This method resets the log page and is called by the log flusher thread
- * after a page has been flushed to disk.
- */
- public void resetLogPage(long lsn, long nextWritePosition, int pageIndex) throws IOException {
-
- String filePath = LogUtil.getLogFilePath(logManagerProperties, getLogFileId(lsn));
-
- logPages[pageIndex].reset(filePath, LogUtil.getFileOffset(this, nextWritePosition), logPageSize);
- }
-
- @Override
- public ILogCursor readLog(PhysicalLogLocator physicalLogLocator, ILogFilter logFilter) throws IOException,
- ACIDException {
- LogCursor cursor = new LogCursor(this, physicalLogLocator, logFilter, logPageSize);
- return cursor;
- }
-
- /*
- * Read a log that is residing on the disk.
- */
- private void readDiskLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException {
- String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, lsnValue));
- long fileOffset = LogUtil.getFileOffset(this, lsnValue);
-
- ByteBuffer buffer = ByteBuffer.allocate(logPageSize);
- RandomAccessFile raf = null;
- FileChannel fileChannel = null;
- try {
- raf = new RandomAccessFile(filePath, "r");
- fileChannel = raf.getChannel();
- fileChannel.position(fileOffset);
- fileChannel.read(buffer);
- buffer.position(0);
-
- byte logType = buffer.get(4);
- int logHeaderSize = logRecordHelper.getLogHeaderSize(logType);
- int logBodySize = buffer.getInt(logHeaderSize - 4);
- int logRecordSize = logHeaderSize + logBodySize + logRecordHelper.getLogChecksumSize();
- buffer.limit(logRecordSize);
- MemBasedBuffer memBuffer = new MemBasedBuffer(buffer.slice());
- if (logicalLogLocator == null) {
- logicalLogLocator = new LogicalLogLocator(lsnValue, memBuffer, 0, this);
- } else {
- logicalLogLocator.setLsn(lsnValue);
- logicalLogLocator.setBuffer(memBuffer);
- logicalLogLocator.setMemoryOffset(0);
- }
- if (!logRecordHelper.validateLogRecord(logicalLogLocator)) {
- throw new ACIDException(" invalid log record at lsn " + lsnValue);
- }
- } catch (Exception fnfe) {
- fnfe.printStackTrace();
- throw new ACIDException(" unable to retrieve log record with lsn " + lsnValue + " from the file system",
- fnfe);
- } finally {
- try {
- if (fileChannel != null) {
- fileChannel.close();
- } else if (raf != null) {
- raf.close();
- }
- } catch (IOException ioe) {
- ioe.printStackTrace();
- throw new ACIDException(" exception in closing a file: " + filePath, ioe);
- }
- }
- }
-
- @Override
- public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException {
- byte[] logRecord = null;
-
- if (lsnValue >= lsn.get()) {
- throw new ACIDException(" invalid lsn " + lsnValue);
- }
-
- /* check if the log record in the log buffer or has reached the disk. */
- if (isMemoryRead(lsnValue)) {
- int pageIndex = getLogPageIndex(lsnValue);
- int pageOffset = getLogPageOffset(lsnValue);
-
- // TODO
- // minimize memory allocation overhead. current code allocates the
- // log page size per reading a log record.
-
- byte[] pageContent = new byte[logPageSize];
-
- // take a lock on the log page so that the page is not flushed to
- // disk interim
- synchronized (logPages[pageIndex]) {
-
- // need to check again (this thread may have got de-scheduled
- // and must refresh!)
-
- if (isMemoryRead(lsnValue)) {
- // get the log record length
- logPages[pageIndex].getBytes(pageContent, 0, pageContent.length);
- byte logType = pageContent[pageOffset + 4];
- int logHeaderSize = logRecordHelper.getLogHeaderSize(logType);
- int logBodySize = DataUtil.byteArrayToInt(pageContent, pageOffset + logHeaderSize - 4);
- int logRecordSize = logHeaderSize + logBodySize + logRecordHelper.getLogChecksumSize();
- logRecord = new byte[logRecordSize];
-
- // copy the log record content
- System.arraycopy(pageContent, pageOffset, logRecord, 0, logRecordSize);
- MemBasedBuffer memBuffer = new MemBasedBuffer(logRecord);
- if (logicalLogLocator == null) {
- logicalLogLocator = new LogicalLogLocator(lsnValue, memBuffer, 0, this);
- } else {
- logicalLogLocator.setLsn(lsnValue);
- logicalLogLocator.setBuffer(memBuffer);
- logicalLogLocator.setMemoryOffset(0);
- }
- try {
- // validate the log record by comparing checksums
- if (!logRecordHelper.validateLogRecord(logicalLogLocator)) {
- throw new ACIDException(" invalid log record at lsn " + lsnValue);
- }
- } catch (Exception e) {
- throw new ACIDException("exception encoutered in validating log record at lsn " + lsnValue, e);
- }
- return;
- }
- }
- }
-
- // the log record is residing on the disk, read it from there.
- readDiskLog(lsnValue, logicalLogLocator);
- }
-
- public boolean isMemoryRead(long currentLSN) {
- long flushLSN = lastFlushedLSN.get();
- if ((flushLSN + 1) == currentLSN) {
- return false;
- }
- long logPageBeginOffset = flushLSN - (flushLSN % logPageSize);
- long logPageEndOffset = logPageBeginOffset + logPageSize;
- if (currentLSN > flushLSN || (currentLSN >= logPageBeginOffset && currentLSN < logPageEndOffset)) {
- return true;
- } else {
- return false;
- }
- }
-
- public void renewLogFiles() throws ACIDException {
- List<String> logFileNames = LogUtil.getLogFiles(logManagerProperties);
- for (String name : logFileNames) {
- File file = new File(LogUtil.getLogFilePath(logManagerProperties, Long.parseLong(name)));
- if (!file.delete()) {
- throw new ACIDException("Failed to delete a file: " + name);
- }
- }
- closeLogPages();
- initLSN();
- openLogPages();
- logPageFlusher.renew();
- }
-
- private void initLSN() throws ACIDException {
- PhysicalLogLocator nextPhysicalLsn = LogUtil.initializeLogAnchor(this);
- startingLSN = nextPhysicalLsn.getLsn();
- lastFlushedLSN.set(startingLSN - 1);
+ appendLSN = initializeLogAnchor(nextLogFileId);
+ flushLSN.set(appendLSN);
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" Starting lsn is : " + startingLSN);
+ LOGGER.info("LogManager starts logging in LSN: " + appendLSN);
}
- lsn.set(startingLSN);
- }
-
- private void closeLogPages() throws ACIDException {
- for (int i = 0; i < numLogPages; i++) {
- try {
- logPages[i].close();
- } catch (IOException e) {
- throw new ACIDException(e);
- }
- }
- }
-
- private void openLogPages() throws ACIDException {
- try {
- String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, startingLSN));
- for (int i = 0; i < numLogPages; i++) {
- logPages[i].open(filePath, LogUtil.getFileOffset(this, startingLSN) + i * logPageSize, logPageSize);
- }
- } catch (Exception e) {
- throw new ACIDException(Thread.currentThread().getName() + " unable to create log buffer", e);
- }
+ appendChannel = getFileChannel(appendLSN, false);
+ getAndInitNewPage();
+ logFlusher = new LogFlusher(this, emptyQ, flushQ);
+ futureLogFlusher = AsterixThreadExecutor.INSTANCE.submit(logFlusher);
}
@Override
- public ILogRecordHelper getLogRecordHelper() {
- return logRecordHelper;
- }
-
- /*
- * Map each log page to cover a physical byte range over a log file. When a
- * page is flushed, the page contents are put to disk in the corresponding
- * byte range.
- */
- private void initializeLogPages(long beginLsn) throws ACIDException {
- try {
- String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, beginLsn));
- long nextDiskWriteOffset = LogUtil.getFileOffset(this, beginLsn);
- long nextBufferWriteOffset = nextDiskWriteOffset % logPageSize;
- long bufferBeginOffset = nextDiskWriteOffset - nextBufferWriteOffset;
-
- for (int i = 0; i < numLogPages; i++) {
- logPages[i] = FileUtil.getFileBasedBuffer(filePath, bufferBeginOffset + i * logPageSize, logPageSize,
- logManagerProperties.getDiskSectorSize());
- if (i == 0) {
- logPages[i].setBufferLastFlushOffset((int) nextBufferWriteOffset);
- logPages[i].setBufferNextWriteOffset((int) nextBufferWriteOffset);
- logPages[i].setDiskNextWriteOffset(nextDiskWriteOffset);
+ public void log(ILogRecord logRecord) throws ACIDException {
+ if (logRecord.getLogSize() > logPageSize) {
+ throw new IllegalStateException();
+ }
+ syncLog(logRecord);
+ if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT)
+ && !logRecord.isFlushed()) {
+ synchronized (logRecord) {
+ while (!logRecord.isFlushed()) {
+ try {
+ logRecord.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
}
}
- } catch (Exception e) {
- e.printStackTrace();
- throw new ACIDException(Thread.currentThread().getName() + " unable to create log buffer", e);
}
}
- /*
- * Pages are sequenced starting with 0 going upto numLogPages-1.
- */
- public int getNextPageInSequence(int pageNo) {
- return (pageNo + 1) % numLogPages;
+ private synchronized void syncLog(ILogRecord logRecord) throws ACIDException {
+ ITransactionContext txnCtx = logRecord.getTxnCtx();
+ if (txnCtx.getTxnState() == ITransactionManager.ABORTED && logRecord.getLogType() != LogType.ABORT) {
+ throw new ACIDException("Aborted job(" + txnCtx.getJobId() + ") tried to write non-abort type log record.");
+ }
+ if (getLogFileOffset(appendLSN) + logRecord.getLogSize() > logFileSize) {
+ prepareNextLogFile();
+ appendPage.isFull(true);
+ getAndInitNewPage();
+ } else if (!appendPage.hasSpace(logRecord.getLogSize())) {
+ appendPage.isFull(true);
+ getAndInitNewPage();
+ }
+ if (logRecord.getLogType() == LogType.UPDATE) {
+ logRecord.setPrevLSN(txnCtx.getLastLSN());
+ }
+ appendPage.append(logRecord, appendLSN);
+ appendLSN += logRecord.getLogSize();
+ }
+
+ private void getAndInitNewPage() {
+ appendPage = null;
+ while (appendPage == null) {
+ try {
+ appendPage = emptyQ.take();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ appendPage.reset();
+ appendPage.setFileChannel(appendChannel);
+ flushQ.offer(appendPage);
}
- public int getPreviousPageInSequence(int pageNo) {
- return pageNo == 0 ? numLogPages - 1 : pageNo - 1;
+ private void prepareNextLogFile() {
+ appendLSN += logFileSize - getLogFileOffset(appendLSN);
+ appendChannel = getFileChannel(appendLSN, true);
+ appendPage.isLastPage(true);
+ //[Notice]
+ //the current log file channel is closed if
+ //LogPage.flush() completely flush the last page of the file.
}
@Override
+ public ILogReader getLogReader(boolean isRecoveryMode) {
+ return new LogReader(this, logFileSize, logPageSize, flushLSN, isRecoveryMode);
+ }
+
public LogManagerProperties getLogManagerProperties() {
return logManagerProperties;
}
- public IFileBasedBuffer getLogPage(int pageIndex) {
- return logPages[pageIndex];
- }
-
- public IFileBasedBuffer[] getLogPages() {
- return logPages;
- }
-
- @Override
public TransactionSubsystem getTransactionSubsystem() {
- return provider;
+ return txnSubsystem;
}
- static AtomicInteger t = new AtomicInteger();
-
- public void decrementActiveTxnCountOnIndexes(int pageIndex) throws HyracksDataException {
- ITransactionContext ctx = null;
- int count = 0;
- int i = 0;
-
- HashMap<ITransactionContext, Integer> map = activeTxnCountMaps.get(pageIndex);
- Set<Map.Entry<ITransactionContext, Integer>> entrySet = map.entrySet();
- if (entrySet != null) {
- for (Map.Entry<ITransactionContext, Integer> entry : entrySet) {
- if (entry != null) {
- if (entry.getValue() != null) {
- count = entry.getValue();
- }
- if (count > 0) {
- ctx = entry.getKey();
- for (i = 0; i < count; i++) {
- ctx.decreaseActiveTransactionCountOnIndexes();
- }
- }
- }
- }
- }
-
- map.clear();
+ public long getAppendLSN() {
+ return appendLSN;
}
@Override
public void start() {
- //no op
+ // no op
}
@Override
public void stop(boolean dumpState, OutputStream os) {
+ terminateLogFlusher();
if (dumpState) {
- //#. dump Configurable Variables
+ // #. dump Configurable Variables
dumpConfVars(os);
- //#. dump LSNInfo
+ // #. dump LSNInfo
dumpLSNInfo(os);
try {
os.flush();
} catch (IOException e) {
- //ignore
+ // ignore
}
}
}
@@ -767,7 +218,7 @@
sb.append("\n>>dump_end\t>>----- [ConfVars] -----\n");
os.write(sb.toString().getBytes());
} catch (Exception e) {
- //ignore exception and continue dumping as much as possible.
+ // ignore exception and continue dumping as much as possible.
if (IS_DEBUG_MODE) {
e.printStackTrace();
}
@@ -778,191 +229,262 @@
try {
StringBuilder sb = new StringBuilder();
sb.append("\n>>dump_begin\t>>----- [LSNInfo] -----");
- sb.append("\nstartingLSN: " + startingLSN);
- sb.append("\ncurrentLSN: " + lsn.get());
- sb.append("\nlastFlushedLSN: " + lastFlushedLSN.get());
+ sb.append("\nappendLsn: " + appendLSN);
+ sb.append("\nflushLsn: " + flushLSN.get());
sb.append("\n>>dump_end\t>>----- [LSNInfo] -----\n");
os.write(sb.toString().getBytes());
} catch (Exception e) {
- //ignore exception and continue dumping as much as possible.
+ // ignore exception and continue dumping as much as possible.
if (IS_DEBUG_MODE) {
e.printStackTrace();
}
}
}
+
+ public MutableLong getFlushLSN() {
+ return flushLSN;
+ }
+
+ private long initializeLogAnchor(long nextLogFileId) {
+ long fileId = 0;
+ long offset = 0;
+ File fileLogDir = new File(logDir);
+ try {
+ if (fileLogDir.exists()) {
+ List<Long> logFileIds = getLogFileIds();
+ if (logFileIds == null) {
+ fileId = nextLogFileId;
+ createFileIfNotExists(getLogFilePath(fileId));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("created a log file: " + getLogFilePath(fileId));
+ }
+ } else {
+ fileId = logFileIds.get(logFileIds.size() - 1);
+ File logFile = new File(getLogFilePath(fileId));
+ offset = logFile.length();
+ }
+ } else {
+ fileId = nextLogFileId;
+ createNewDirectory(logDir);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
+ }
+ createFileIfNotExists(getLogFilePath(fileId));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("created a log file: " + getLogFilePath(fileId));
+ }
+ }
+ } catch (IOException ioe) {
+ throw new IllegalStateException("Failed to initialize the log anchor", ioe);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("log file Id: " + fileId + ", offset: " + offset);
+ }
+ return logFileSize * fileId + offset;
+ }
+
+ public void renewLogFiles() {
+ terminateLogFlusher();
+ long lastMaxLogFileId = deleteAllLogFiles();
+ initializeLogManager(lastMaxLogFileId + 1);
+ }
+
+ private void terminateLogFlusher() {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Terminating LogFlusher thread ...");
+ }
+ logFlusher.terminate();
+ try {
+ futureLogFlusher.get();
+ } catch (ExecutionException | InterruptedException e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("---------- warning(begin): LogFlusher thread is terminated abnormally --------");
+ e.printStackTrace();
+ LOGGER.info("---------- warning(end) : LogFlusher thread is terminated abnormally --------");
+ }
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("LogFlusher thread is terminated.");
+ }
+ }
+
+ private long deleteAllLogFiles() {
+ if (appendChannel != null) {
+ try {
+ appendChannel.close();
+ } catch (IOException e) {
+ throw new IllegalStateException("Failed to close a fileChannel of a log file");
+ }
+ }
+ List<Long> logFileIds = getLogFileIds();
+ for (Long id : logFileIds) {
+ File file = new File(getLogFilePath(id));
+ if (!file.delete()) {
+ throw new IllegalStateException("Failed to delete a file: " + file.getAbsolutePath());
+ }
+ }
+ return logFileIds.get(logFileIds.size() - 1);
+ }
+
+ private List<Long> getLogFileIds() {
+ File fileLogDir = new File(logDir);
+ String[] logFileNames = null;
+ List<Long> logFileIds = null;
+ if (fileLogDir.exists()) {
+ logFileNames = fileLogDir.list(new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ if (name.startsWith(logFilePrefix)) {
+ return true;
+ }
+ return false;
+ }
+ });
+ if (logFileNames != null && logFileNames.length != 0) {
+ logFileIds = new ArrayList<Long>();
+ for (String fileName : logFileNames) {
+ logFileIds.add(Long.parseLong(fileName.substring(logFilePrefix.length() + 1)));
+ }
+ Collections.sort(logFileIds, new Comparator<Long>() {
+ @Override
+ public int compare(Long arg0, Long arg1) {
+ return arg0.compareTo(arg1);
+ }
+ });
+ }
+ }
+ return logFileIds;
+ }
+
+ public String getLogFilePath(long fileId) {
+ return logDir + File.separator + logFilePrefix + "_" + fileId;
+ }
+
+ public long getLogFileOffset(long lsn) {
+ return lsn % logFileSize;
+ }
+
+ public long getLogFileId(long lsn) {
+ return lsn / logFileSize;
+ }
+
+ private boolean createFileIfNotExists(String path) throws IOException {
+ File file = new File(path);
+ File parentFile = file.getParentFile();
+ if (parentFile != null) {
+ parentFile.mkdirs();
+ }
+ return file.createNewFile();
+ }
+
+ private boolean createNewDirectory(String path) throws IOException {
+ return (new File(path)).mkdir();
+ }
+
+ public FileChannel getFileChannel(long lsn, boolean create) {
+ FileChannel newFileChannel = null;
+ try {
+ long fileId = getLogFileId(lsn);
+ String logFilePath = getLogFilePath(fileId);
+ File file = new File(logFilePath);
+ if (create) {
+ if (!file.createNewFile()) {
+ throw new IllegalStateException();
+ }
+ } else {
+ if (!file.exists()) {
+ throw new IllegalStateException();
+ }
+ }
+ RandomAccessFile raf = new RandomAccessFile(new File(logFilePath), "rw");
+ newFileChannel = raf.getChannel();
+ newFileChannel.position(getLogFileOffset(lsn));
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ return newFileChannel;
+ }
+
+ public long getReadableSmallestLSN() {
+ List<Long> logFileIds = getLogFileIds();
+ return logFileIds.get(0) * logFileSize;
+ }
}
-/*
- * The thread responsible for putting log pages to disk in an ordered manner.
- * The Log Flusher updates the bookkeeping data internal to the log manager and
- * acquires appropriate locks. It also acquires finer level locks on the log
- * page when it is in process of flushing the content to disk.
- */
-class LogPageFlushThread extends Thread {
+class LogFlusher implements Callable<Boolean> {
+ private static final Logger LOGGER = Logger.getLogger(LogFlusher.class.getName());
+ private final static LogPage POISON_PILL = new LogPage(null, ILogRecord.JOB_TERMINATE_LOG_SIZE, null);
+ private final LogManager logMgr;//for debugging
+ private final LinkedBlockingQueue<LogPage> emptyQ;
+ private final LinkedBlockingQueue<LogPage> flushQ;
+ private LogPage flushPage;
+ private final AtomicBoolean isStarted;
+ private final AtomicBoolean terminateFlag;
- private LogManager logManager;
- /*
- * pendingFlushRequests is a map with key as Integer denoting the page
- * index. When a (transaction) thread discovers the need to flush a page, it
- * puts its Thread object into the corresponding value that is a
- * LinkedBlockingQueue. The LogManager has a LogFlusher thread that scans
- * this map in order of page index (and circling around). The flusher thread
- * needs to flush pages in order and waits for a thread to deposit an object
- * in the blocking queue corresponding to the next page in order. A request
- * to flush a page is conveyed to the flush thread by simply depositing an
- * object in to corresponding blocking queue. It is blocking in the sense
- * that the flusher thread will continue to wait for an object to arrive in
- * the queue. The object itself is ignored by the fliusher and just acts as
- * a signal/event that a page needs to be flushed.
- */
- private final LinkedBlockingQueue<Object>[] flushRequestQueue;
- private final Object[] flushRequests;
- private int flushPageIndex;
- private final long groupCommitWaitPeriod;
- private boolean isRenewRequest;
+ public LogFlusher(LogManager logMgr, LinkedBlockingQueue<LogPage> emptyQ, LinkedBlockingQueue<LogPage> flushQ) {
+ this.logMgr = logMgr;
+ this.emptyQ = emptyQ;
+ this.flushQ = flushQ;
+ flushPage = null;
+ isStarted = new AtomicBoolean(false);
+ terminateFlag = new AtomicBoolean(false);
- public LogPageFlushThread(LogManager logManager) {
- this.logManager = logManager;
- setName("Flusher");
- int numLogPages = logManager.getLogManagerProperties().getNumLogPages();
- this.flushRequestQueue = new LinkedBlockingQueue[numLogPages];
- this.flushRequests = new Object[numLogPages];
- for (int i = 0; i < numLogPages; i++) {
- flushRequestQueue[i] = new LinkedBlockingQueue<Object>(1);
- flushRequests[i] = new Object();
- }
- this.flushPageIndex = 0;
- groupCommitWaitPeriod = logManager.getLogManagerProperties().getGroupCommitWaitPeriod();
- isRenewRequest = false;
}
- public void renew() {
- isRenewRequest = true;
- flushPageIndex = 0;
- this.interrupt();
- isRenewRequest = false;
- }
-
- public void requestFlush(int pageIndex, long lsn, boolean isSynchronous) {
- synchronized (logManager.getLogPage(pageIndex)) {
- // return if flushedLSN >= lsn
- if (logManager.getLastFlushedLsn().get() >= lsn) {
- return;
- }
-
- // put a new request to the queue only if the request on the page is
- // not in the queue.
- flushRequestQueue[pageIndex].offer(flushRequests[pageIndex]);
-
- // return if the request is asynchronous
- if (!isSynchronous) {
- return;
- }
-
- // wait until there is flush.
- boolean isNotified = false;
- while (!isNotified) {
+ public void terminate() {
+ //make sure the LogFlusher thread started before terminating it.
+ synchronized (isStarted) {
+ while (!isStarted.get()) {
try {
- logManager.getLogPage(pageIndex).wait();
- isNotified = true;
+ isStarted.wait();
} catch (InterruptedException e) {
- e.printStackTrace();
+ //ignore
}
}
}
+
+ terminateFlag.set(true);
+ if (flushPage != null) {
+ synchronized (flushPage) {
+ flushPage.isStop(true);
+ flushPage.notify();
+ }
+ }
+ //[Notice]
+ //The return value doesn't need to be checked
+ //since terminateFlag will trigger termination if the flushQ is full.
+ flushQ.offer(POISON_PILL);
}
@Override
- public void run() {
- int logPageSize = logManager.getLogManagerProperties().getLogPageSize();
- int logBufferSize = logManager.getLogManagerProperties().getLogBufferSize();
- int beforeFlushOffset = 0;
- int afterFlushOffset = 0;
- boolean resetFlushPageIndex = false;
-
- while (true) {
- try {
- // A wait call on the linkedBLockingQueue. The flusher thread is
- // notified when an object is added to the queue. Please note
- // that each page has an associated blocking queue.
+ public Boolean call() {
+ synchronized (isStarted) {
+ isStarted.set(true);
+ isStarted.notify();
+ }
+ try {
+ while (true) {
+ flushPage = null;
try {
- flushRequestQueue[flushPageIndex].take();
- } catch (InterruptedException ie) {
- while (isRenewRequest) {
- sleep(1);
+ flushPage = flushQ.take();
+ if (flushPage == POISON_PILL || terminateFlag.get()) {
+ return true;
}
- continue;
- }
-
- //if the log page is already full, don't wait.
- if (logManager.getLogPage(flushPageIndex).getBufferNextWriteOffset() < logPageSize
- - logManager.getLogRecordHelper().getCommitLogSize()) {
- // #. sleep for the groupCommitWaitTime
- sleep(groupCommitWaitPeriod);
- }
-
- synchronized (logManager.getLogPage(flushPageIndex)) {
- logManager.getLogPage(flushPageIndex).acquireWriteLatch();
- try {
-
- // #. need to wait until the reference count reaches 0
- while (logManager.getLogPage(flushPageIndex).getRefCnt() != 0) {
- sleep(0);
- }
-
- beforeFlushOffset = logManager.getLogPage(flushPageIndex).getBufferLastFlushOffset();
-
- // put the content to disk (the thread still has a lock on the log page)
- logManager.getLogPage(flushPageIndex).flush();
-
- afterFlushOffset = logManager.getLogPage(flushPageIndex).getBufferLastFlushOffset();
-
- // increment the last flushed lsn
- logManager.incrementLastFlushedLsn(afterFlushOffset - beforeFlushOffset);
-
- // increment currentLSN if currentLSN is less than flushLSN.
- if (logManager.getLastFlushedLsn().get() + 1 > logManager.getCurrentLsn().get()) {
- logManager.getCurrentLsn().set(logManager.getLastFlushedLsn().get() + 1);
- }
-
- // Map the log page to a new region in the log file if the flushOffset reached the logPageSize
- if (afterFlushOffset == logPageSize) {
- long diskNextWriteOffset = logManager.getLogPages()[flushPageIndex]
- .getDiskNextWriteOffset() + logBufferSize;
- logManager.resetLogPage(logManager.getLastFlushedLsn().get() + 1 + logBufferSize,
- diskNextWriteOffset, flushPageIndex);
- resetFlushPageIndex = true;
- }
-
- // decrement activeTxnCountOnIndexes
- logManager.decrementActiveTxnCountOnIndexes(flushPageIndex);
-
- } finally {
- logManager.getLogPage(flushPageIndex).releaseWriteLatch();
- }
-
- // #. checks the queue whether there is another flush
- // request on the same log buffer
- // If there is another request, then simply remove it.
- if (flushRequestQueue[flushPageIndex].peek() != null) {
- flushRequestQueue[flushPageIndex].take();
- }
-
- // notify all waiting (transaction) threads.
- logManager.getLogPage(flushPageIndex).notifyAll();
-
- if (resetFlushPageIndex) {
- flushPageIndex = logManager.getNextPageInSequence(flushPageIndex);
- resetFlushPageIndex = false;
+ } catch (InterruptedException e) {
+ if (flushPage == null) {
+ continue;
}
}
- } catch (IOException ioe) {
- ioe.printStackTrace();
- throw new Error(" exception in flushing log page", ioe);
- } catch (InterruptedException e) {
- e.printStackTrace();
- break;
+ flushPage.flush();
+ emptyQ.offer(flushPage);
}
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("-------------------------------------------------------------------------");
+ LOGGER.info("LogFlusher is terminating abnormally. System is in unusalbe state.");
+ LOGGER.info("-------------------------------------------------------------------------");
+ }
+ e.printStackTrace();
+ throw e;
}
}
}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
new file mode 100644
index 0000000..a3f42a7
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogPage;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
+
+public class LogPage implements ILogPage {
+
+ public static final boolean IS_DEBUG_MODE = false;//true
+ private static final Logger LOGGER = Logger.getLogger(LogPage.class.getName());
+ private final LockManager lockMgr;
+ private final LogPageReader logPageReader;
+ private final int logPageSize;
+ private final MutableLong flushLSN;
+ private final AtomicBoolean full;
+ private int appendOffset;
+ private int flushOffset;
+ private final ByteBuffer appendBuffer;
+ private final ByteBuffer flushBuffer;
+ private final ByteBuffer unlockBuffer;
+ private boolean isLastPage;
+ private final LinkedBlockingQueue<ILogRecord> syncCommitQ;
+ private FileChannel fileChannel;
+ private boolean stop;
+
+ public LogPage(LockManager lockMgr, int logPageSize, MutableLong flushLSN) {
+ this.lockMgr = lockMgr;
+ this.logPageSize = logPageSize;
+ this.flushLSN = flushLSN;
+ appendBuffer = ByteBuffer.allocate(logPageSize);
+ flushBuffer = appendBuffer.duplicate();
+ unlockBuffer = appendBuffer.duplicate();
+ logPageReader = getLogPageReader();
+ full = new AtomicBoolean(false);
+ appendOffset = 0;
+ flushOffset = 0;
+ isLastPage = false;
+ syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.JOB_TERMINATE_LOG_SIZE);
+ }
+
+ ////////////////////////////////////
+ // LogAppender Methods
+ ////////////////////////////////////
+
+ @Override
+ public void append(ILogRecord logRecord, long appendLSN) {
+ logRecord.writeLogRecord(appendBuffer);
+ logRecord.getTxnCtx().setLastLSN(logRecord.getLogType() == LogType.UPDATE ? logRecord.getResourceId() : -1,
+ appendLSN);
+ synchronized (this) {
+ appendOffset += logRecord.getLogSize();
+ if (IS_DEBUG_MODE) {
+ LOGGER.info("append()| appendOffset: " + appendOffset);
+ }
+ if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+ logRecord.isFlushed(false);
+ syncCommitQ.offer(logRecord);
+ }
+ this.notify();
+ }
+ }
+
+ public void setFileChannel(FileChannel fileChannel) {
+ this.fileChannel = fileChannel;
+ }
+
+ public void setInitialFlushOffset(long offset) {
+ try {
+ fileChannel.position(offset);
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ public synchronized void isFull(boolean full) {
+ this.full.set(full);
+ this.notify();
+ }
+
+ public void isLastPage(boolean isLastPage) {
+ this.isLastPage = isLastPage;
+ }
+
+ public boolean hasSpace(int logSize) {
+ return appendOffset + logSize <= logPageSize;
+ }
+
+ public void reset() {
+ appendBuffer.position(0);
+ appendBuffer.limit(logPageSize);
+ flushBuffer.position(0);
+ flushBuffer.limit(logPageSize);
+ unlockBuffer.position(0);
+ unlockBuffer.limit(logPageSize);
+ full.set(false);
+ appendOffset = 0;
+ flushOffset = 0;
+ isLastPage = false;
+ }
+
+ ////////////////////////////////////
+ // LogFlusher Methods
+ ////////////////////////////////////
+
+ @Override
+ public void flush() {
+ try {
+ int endOffset;
+ while (!full.get()) {
+ synchronized (this) {
+ if (appendOffset - flushOffset == 0 && !full.get()) {
+ try {
+ if (IS_DEBUG_MODE) {
+ LOGGER.info("flush()| appendOffset: " + appendOffset + ", flushOffset: " + flushOffset
+ + ", full: " + full.get());
+ }
+ if (stop) {
+ fileChannel.close();
+ break;
+ }
+ this.wait();
+ } catch (InterruptedException e) {
+ continue;
+ }
+ }
+ endOffset = appendOffset;
+ }
+ internalFlush(flushOffset, endOffset);
+ }
+ internalFlush(flushOffset, appendOffset);
+ if (isLastPage) {
+ fileChannel.close();
+ }
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ private void internalFlush(int beginOffset, int endOffset) {
+ try {
+ if (endOffset > beginOffset) {
+ flushBuffer.limit(endOffset);
+ fileChannel.write(flushBuffer);
+ fileChannel.force(false);
+ flushOffset = endOffset;
+ synchronized (flushLSN) {
+ flushLSN.set(flushLSN.get() + (endOffset - beginOffset));
+ flushLSN.notifyAll(); //notify to LogReaders if any
+ }
+ if (IS_DEBUG_MODE) {
+ LOGGER.info("internalFlush()| flushOffset: " + flushOffset + ", flushLSN: " + flushLSN.get());
+ }
+ batchUnlock(beginOffset, endOffset);
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ private LogPageReader getLogPageReader() {
+ return new LogPageReader(unlockBuffer);
+ }
+
+ private void batchUnlock(int beginOffset, int endOffset) throws ACIDException {
+ if (endOffset > beginOffset) {
+ logPageReader.initializeScan(beginOffset, endOffset);
+ lockMgr.batchUnlock(this, logPageReader);
+ }
+ }
+
+ public void notifyJobTerminator() {
+ ILogRecord logRecord = null;
+ while (logRecord == null) {
+ try {
+ logRecord = syncCommitQ.take();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ synchronized (logRecord) {
+ logRecord.isFlushed(true);
+ logRecord.notifyAll();
+ }
+ }
+
+ public boolean isStop() {
+ return stop;
+ }
+
+ public void isStop(boolean stop) {
+ this.stop = stop;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java
new file mode 100644
index 0000000..9e54abc
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.nio.ByteBuffer;
+
+public class LogPageReader {
+
+ private final ByteBuffer buffer;
+ private final LogRecord logRecord;
+ private int endOffset;
+
+ public LogPageReader(ByteBuffer buffer) {
+ this.buffer = buffer;
+ logRecord = new LogRecord();
+ }
+
+ public void initializeScan(int beginOffset, int endOffset) {
+ this.endOffset = endOffset;
+ buffer.position(beginOffset);
+ }
+
+ public LogRecord next() {
+ if (buffer.position() == endOffset) {
+ return null;
+ }
+ if (!logRecord.readLogRecord(buffer)) {
+ throw new IllegalStateException();
+ }
+ return logRecord;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java
new file mode 100644
index 0000000..9dc966c
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java
@@ -0,0 +1,188 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+
+public class LogReader implements ILogReader {
+
+ public static final boolean IS_DEBUG_MODE = false;//true
+ private static final Logger LOGGER = Logger.getLogger(LogReader.class.getName());
+ private final LogManager logMgr;
+ private final long logFileSize;
+ private final int logPageSize;
+ private final MutableLong flushLSN;
+ private final boolean isRecoveryMode;
+ private final ByteBuffer readBuffer;
+ private final ILogRecord logRecord;
+ private long readLSN;
+ private long bufferBeginLSN;
+ private long fileBeginLSN;
+ private FileChannel fileChannel;
+
+ private enum ReturnState {
+ FLUSH,
+ EOF
+ };
+
+ public LogReader(LogManager logMgr, long logFileSize, int logPageSize, MutableLong flushLSN, boolean isRecoveryMode) {
+ this.logMgr = logMgr;
+ this.logFileSize = logFileSize;
+ this.logPageSize = logPageSize;
+ this.flushLSN = flushLSN;
+ this.isRecoveryMode = isRecoveryMode;
+ this.readBuffer = ByteBuffer.allocate(logPageSize);
+ this.logRecord = new LogRecord();
+ }
+
+ @Override
+ public void initializeScan(long beginLSN) throws ACIDException {
+ readLSN = beginLSN;
+ if (waitForFlushOrReturnIfEOF() == ReturnState.EOF) {
+ return;
+ }
+ getFileChannel();
+ readPage();
+ }
+
+ //for scanning
+ @Override
+ public ILogRecord next() throws ACIDException {
+ if (waitForFlushOrReturnIfEOF() == ReturnState.EOF) {
+ return null;
+ }
+ if (readBuffer.position() == readBuffer.limit() || !logRecord.readLogRecord(readBuffer)) {
+ readNextPage();
+ if (!logRecord.readLogRecord(readBuffer)) {
+ throw new IllegalStateException();
+ }
+ }
+ logRecord.setLSN(readLSN);
+ readLSN += logRecord.getLogSize();
+ return logRecord;
+ }
+
+ private ReturnState waitForFlushOrReturnIfEOF() {
+ synchronized (flushLSN) {
+ while (readLSN >= flushLSN.get()) {
+ if (isRecoveryMode) {
+ return ReturnState.EOF;
+ }
+ try {
+ if (IS_DEBUG_MODE) {
+ LOGGER.info("waitForFlushOrReturnIfEOF()| flushLSN: " + flushLSN.get() + ", readLSN: "
+ + readLSN);
+ }
+ flushLSN.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ return ReturnState.FLUSH;
+ }
+ }
+
+ private void readNextPage() throws ACIDException {
+ try {
+ if (readLSN % logFileSize == fileChannel.size()) {
+ fileChannel.close();
+ readLSN += logFileSize - (readLSN % logFileSize);
+ getFileChannel();
+ }
+ readPage();
+ } catch (IOException e) {
+ throw new ACIDException(e);
+ }
+ }
+
+ private void readPage() throws ACIDException {
+ int size;
+ readBuffer.position(0);
+ readBuffer.limit(logPageSize);
+ try {
+ fileChannel.position(readLSN % logFileSize);
+ size = fileChannel.read(readBuffer);
+ } catch (IOException e) {
+ throw new ACIDException(e);
+ }
+ readBuffer.position(0);
+ readBuffer.limit(size);
+ bufferBeginLSN = readLSN;
+ }
+
+ //for random reading
+ @Override
+ public ILogRecord read(long LSN) throws ACIDException {
+ readLSN = LSN;
+ synchronized (flushLSN) {
+ while (readLSN >= flushLSN.get()) {
+ try {
+ flushLSN.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+ try {
+ if (fileChannel == null) {
+ getFileChannel();
+ readPage();
+ } else if (readLSN < fileBeginLSN || readLSN >= fileBeginLSN + fileChannel.size()) {
+ fileChannel.close();
+ getFileChannel();
+ readPage();
+ } else if (readLSN < bufferBeginLSN || readLSN >= bufferBeginLSN + readBuffer.limit()) {
+ readPage();
+ } else {
+ readBuffer.position((int) (readLSN - bufferBeginLSN));
+ }
+ } catch (IOException e) {
+ throw new ACIDException(e);
+ }
+ if (!logRecord.readLogRecord(readBuffer)) {
+ readNextPage();
+ if (!logRecord.readLogRecord(readBuffer)) {
+ throw new IllegalStateException();
+ }
+ }
+ logRecord.setLSN(readLSN);
+ readLSN += logRecord.getLogSize();
+ return logRecord;
+ }
+
+ private void getFileChannel() throws ACIDException {
+ fileChannel = logMgr.getFileChannel(readLSN, false);
+ fileBeginLSN = readLSN;
+ }
+
+ @Override
+ public void close() throws ACIDException {
+ try {
+ if (fileChannel != null) {
+ fileChannel.close();
+ }
+ } catch (IOException e) {
+ throw new ACIDException(e);
+ }
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
new file mode 100644
index 0000000..dd81df7
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
@@ -0,0 +1,526 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.zip.CRC32;
+
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
+
+/*
+ * == LogRecordFormat ==
+ * ---------------------------
+ * [Header1] (5 bytes) : for all log types
+ * LogType(1)
+ * JobId(4)
+ * ---------------------------
+ * [Header2] (12 bytes + PKValueSize) : for entity_commit and update log types
+ * DatasetId(4) //stored in dataset_dataset in Metadata Node
+ * PKHashValue(4)
+ * PKValueSize(4)
+ * PKValue(PKValueSize)
+ * ---------------------------
+ * [Header3] (21 bytes) : only for update log type
+ * PrevLSN(8)
+ * ResourceId(8) //stored in .metadata of the corresponding index in NC node
+ * ResourceType(1)
+ * LogRecordSize(4)
+ * ---------------------------
+ * [Body] (Variable size) : only for update log type
+ * FieldCnt(4)
+ * NewOp(1)
+ * NewValueSize(4)
+ * NewValue(NewValueSize)
+ * OldOp(1)
+ * OldValueSize(4)
+ * OldValue(OldValueSize)
+ * ---------------------------
+ * [Tail] (8 bytes) : for all log types
+ * Checksum(8)
+ * ---------------------------
+ * = LogSize =
+ * 1) JOB_COMMIT_LOG_SIZE: 13 bytes (5 + 8)
+ * 2) ENTITY_COMMIT: 25 + PKSize (5 + 12 + PKSize + 8)
+ * --> ENTITY_COMMIT_LOG_BASE_SIZE = 25
+ * 3) UPDATE: 64 + PKSize + New/OldValueSize (5 + 12 + PKSize + 21 + 14 + New/OldValueSize + 8)
+ * --> UPDATE_LOG_BASE_SIZE = 60
+ */
+public class LogRecord implements ILogRecord {
+
+ //------------- fields in a log record (begin) ------------//
+ private byte logType;
+ private int jobId;
+ private int datasetId;
+ private int PKHashValue;
+ private int PKValueSize;
+ private ITupleReference PKValue;
+ private long prevLSN;
+ private long resourceId;
+ private byte resourceType;
+ private int logSize;
+ private int fieldCnt;
+ private byte newOp;
+ private int newValueSize;
+ private ITupleReference newValue;
+ private byte oldOp;
+ private int oldValueSize;
+ private ITupleReference oldValue;
+ private long checksum;
+ //------------- fields in a log record (end) --------------//
+
+ private int PKFieldCnt;
+ private static final int CHECKSUM_SIZE = 8;
+ private ITransactionContext txnCtx;
+ private long LSN;
+ private final AtomicBoolean isFlushed;
+ private final SimpleTupleWriter tupleWriter;
+ private final PrimaryKeyTupleReference readPKValue;
+ private final SimpleTupleReference readNewValue;
+ private final SimpleTupleReference readOldValue;
+ private final CRC32 checksumGen;
+ private int[] PKFields;
+
+ public LogRecord() {
+ isFlushed = new AtomicBoolean(false);
+ tupleWriter = new SimpleTupleWriter();
+ readPKValue = new PrimaryKeyTupleReference();
+ readNewValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+ readOldValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+ checksumGen = new CRC32();
+ }
+
+ @Override
+ public void writeLogRecord(ByteBuffer buffer) {
+ int beginOffset = buffer.position();
+ buffer.put(logType);
+ buffer.putInt(jobId);
+ if (logType == LogType.UPDATE || logType == LogType.ENTITY_COMMIT) {
+ buffer.putInt(datasetId);
+ buffer.putInt(PKHashValue);
+ if (PKValueSize <= 0) {
+ throw new IllegalStateException("Primary Key Size is less than or equal to 0");
+ }
+ buffer.putInt(PKValueSize);
+ writePKValue(buffer);
+ }
+ if (logType == LogType.UPDATE) {
+ buffer.putLong(prevLSN);
+ buffer.putLong(resourceId);
+ buffer.put(resourceType);
+ buffer.putInt(logSize);
+ buffer.putInt(fieldCnt);
+ buffer.put(newOp);
+ buffer.putInt(newValueSize);
+ writeTuple(buffer, newValue, newValueSize);
+ if (resourceType == ResourceType.LSM_BTREE) {
+ buffer.put(oldOp);
+ if (oldOp != (byte) (IndexOperation.NOOP.ordinal())) {
+ buffer.putInt(oldValueSize);
+ if (oldValueSize > 0) {
+ writeTuple(buffer, oldValue, oldValueSize);
+ }
+ }
+ }
+ }
+ checksum = generateChecksum(buffer, beginOffset, logSize - CHECKSUM_SIZE);
+ buffer.putLong(checksum);
+ }
+
+ private void writePKValue(ByteBuffer buffer) {
+ int i;
+ for (i = 0; i < PKFieldCnt; i++) {
+ buffer.put(PKValue.getFieldData(0), PKValue.getFieldStart(PKFields[i]), PKValue.getFieldLength(PKFields[i]));
+ }
+ }
+
+ private void writeTuple(ByteBuffer buffer, ITupleReference tuple, int size) {
+ tupleWriter.writeTuple(tuple, buffer.array(), buffer.position());
+ //writeTuple() doesn't change the position of the buffer.
+ buffer.position(buffer.position() + size);
+ }
+
+ private long generateChecksum(ByteBuffer buffer, int offset, int len) {
+ checksumGen.reset();
+ checksumGen.update(buffer.array(), offset, len);
+ return checksumGen.getValue();
+ }
+
+ @Override
+ public boolean readLogRecord(ByteBuffer buffer) {
+ int beginOffset = buffer.position();
+ try {
+ logType = buffer.get();
+ jobId = buffer.getInt();
+ if (logType == LogType.JOB_COMMIT || logType == LogType.ABORT) {
+ datasetId = -1;
+ PKHashValue = -1;
+ } else {
+ datasetId = buffer.getInt();
+ PKHashValue = buffer.getInt();
+ PKValueSize = buffer.getInt();
+ if (PKValueSize <= 0) {
+ throw new IllegalStateException("Primary Key Size is less than or equal to 0");
+ }
+ PKValue = readPKValue(buffer);
+ }
+ if (logType == LogType.UPDATE) {
+ prevLSN = buffer.getLong();
+ resourceId = buffer.getLong();
+ resourceType = buffer.get();
+ logSize = buffer.getInt();
+ fieldCnt = buffer.getInt();
+ newOp = buffer.get();
+ newValueSize = buffer.getInt();
+ newValue = readTuple(buffer, readNewValue, fieldCnt, newValueSize);
+ if (resourceType == ResourceType.LSM_BTREE) {
+ oldOp = buffer.get();
+ if (oldOp != (byte) (IndexOperation.NOOP.ordinal())) {
+ oldValueSize = buffer.getInt();
+ if (oldValueSize > 0) {
+ oldValue = readTuple(buffer, readOldValue, fieldCnt, oldValueSize);
+ }
+ }
+ }
+ } else {
+ computeAndSetLogSize();
+ }
+ checksum = buffer.getLong();
+ if (checksum != generateChecksum(buffer, beginOffset, logSize - CHECKSUM_SIZE)) {
+ throw new IllegalStateException();
+ }
+ } catch (BufferUnderflowException e) {
+ buffer.position(beginOffset);
+ return false;
+ }
+ return true;
+ }
+
+ private ITupleReference readPKValue(ByteBuffer buffer) {
+ if (buffer.position() + PKValueSize > buffer.limit()) {
+ throw new BufferUnderflowException();
+ }
+ readPKValue.reset(buffer.array(), buffer.position(), PKValueSize);
+ buffer.position(buffer.position() + PKValueSize);
+ return readPKValue;
+ }
+
+ private ITupleReference readTuple(ByteBuffer srcBuffer, SimpleTupleReference destTuple, int fieldCnt, int size) {
+ if (srcBuffer.position() + size > srcBuffer.limit()) {
+ throw new BufferUnderflowException();
+ }
+ destTuple.setFieldCount(fieldCnt);
+ destTuple.resetByTupleOffset(srcBuffer, srcBuffer.position());
+ srcBuffer.position(srcBuffer.position() + size);
+ return destTuple;
+ }
+
+ @Override
+ public void formJobTerminateLogRecord(ITransactionContext txnCtx, boolean isCommit) {
+ this.txnCtx = txnCtx;
+ this.logType = isCommit ? LogType.JOB_COMMIT : LogType.ABORT;
+ this.jobId = txnCtx.getJobId().getId();
+ this.datasetId = -1;
+ this.PKHashValue = -1;
+ computeAndSetLogSize();
+ }
+
+ @Override
+ public void formEntityCommitLogRecord(ITransactionContext txnCtx, int datasetId, int PKHashValue,
+ ITupleReference PKValue, int[] PKFields) {
+ this.txnCtx = txnCtx;
+ this.logType = LogType.ENTITY_COMMIT;
+ this.jobId = txnCtx.getJobId().getId();
+ this.datasetId = datasetId;
+ this.PKHashValue = PKHashValue;
+ this.PKFieldCnt = PKFields.length;
+ this.PKValue = PKValue;
+ this.PKFields = PKFields;
+ computeAndSetPKValueSize();
+ computeAndSetLogSize();
+ }
+
+ @Override
+ public void computeAndSetPKValueSize() {
+ int i;
+ PKValueSize = 0;
+ for (i = 0; i < PKFieldCnt; i++) {
+ PKValueSize += PKValue.getFieldLength(PKFields[i]);
+ }
+ }
+
+ private void setUpdateLogSize() {
+ logSize = UPDATE_LOG_BASE_SIZE + PKValueSize + newValueSize + oldValueSize;
+ if (resourceType != ResourceType.LSM_BTREE) {
+ logSize -= 5; //oldOp(byte: 1) + oldValueLength(int: 4)
+ } else {
+ if (oldOp == (byte) (IndexOperation.NOOP.ordinal())) {
+ logSize -= 4; //oldValueLength(int: 4)
+ }
+ }
+ }
+
+ @Override
+ public void computeAndSetLogSize() {
+ switch (logType) {
+ case LogType.UPDATE:
+ setUpdateLogSize();
+ break;
+ case LogType.JOB_COMMIT:
+ case LogType.ABORT:
+ logSize = JOB_TERMINATE_LOG_SIZE;
+ break;
+ case LogType.ENTITY_COMMIT:
+ logSize = ENTITY_COMMIT_LOG_BASE_SIZE + PKValueSize;
+ break;
+ default:
+ throw new IllegalStateException("Unsupported Log Type");
+ }
+ }
+
+ @Override
+ public String getLogRecordForDisplay() {
+ StringBuilder builder = new StringBuilder();
+ builder.append(" LSN : ").append(LSN);
+ builder.append(" LogType : ").append(LogType.toString(logType));
+ builder.append(" LogSize : ").append(logSize);
+ builder.append(" JobId : ").append(jobId);
+ if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPDATE) {
+ builder.append(" DatasetId : ").append(datasetId);
+ builder.append(" PKHashValue : ").append(PKHashValue);
+ builder.append(" PKFieldCnt : ").append(PKFieldCnt);
+ builder.append(" PKSize: ").append(PKValueSize);
+ }
+ if (logType == LogType.UPDATE) {
+ builder.append(" PrevLSN : ").append(prevLSN);
+ builder.append(" ResourceId : ").append(resourceId);
+ builder.append(" ResourceType : ").append(resourceType);
+ }
+ return builder.toString();
+ }
+
+ ////////////////////////////////////////////
+ // getter and setter methods
+ ////////////////////////////////////////////
+
+ @Override
+ public ITransactionContext getTxnCtx() {
+ return txnCtx;
+ }
+
+ @Override
+ public void setTxnCtx(ITransactionContext txnCtx) {
+ this.txnCtx = txnCtx;
+ }
+
+ @Override
+ public boolean isFlushed() {
+ return isFlushed.get();
+ }
+
+ @Override
+ public void isFlushed(boolean isFlushed) {
+ this.isFlushed.set(isFlushed);
+ }
+
+ @Override
+ public byte getLogType() {
+ return logType;
+ }
+
+ @Override
+ public void setLogType(byte logType) {
+ this.logType = logType;
+ }
+
+ @Override
+ public int getJobId() {
+ return jobId;
+ }
+
+ @Override
+ public void setJobId(int jobId) {
+ this.jobId = jobId;
+ }
+
+ @Override
+ public int getDatasetId() {
+ return datasetId;
+ }
+
+ @Override
+ public void setDatasetId(int datasetId) {
+ this.datasetId = datasetId;
+ }
+
+ @Override
+ public int getPKHashValue() {
+ return PKHashValue;
+ }
+
+ @Override
+ public void setPKHashValue(int PKHashValue) {
+ this.PKHashValue = PKHashValue;
+ }
+
+ @Override
+ public long getPrevLSN() {
+ return prevLSN;
+ }
+
+ @Override
+ public void setPrevLSN(long prevLSN) {
+ this.prevLSN = prevLSN;
+ }
+
+ @Override
+ public long getResourceId() {
+ return resourceId;
+ }
+
+ @Override
+ public void setResourceId(long resourceId) {
+ this.resourceId = resourceId;
+ }
+
+ @Override
+ public byte getResourceType() {
+ return resourceType;
+ }
+
+ @Override
+ public void setResourceType(byte resourceType) {
+ this.resourceType = resourceType;
+ }
+
+ @Override
+ public int getLogSize() {
+ return logSize;
+ }
+
+ @Override
+ public void setLogSize(int logSize) {
+ this.logSize = logSize;
+ }
+
+ @Override
+ public byte getNewOp() {
+ return newOp;
+ }
+
+ @Override
+ public void setNewOp(byte newOp) {
+ this.newOp = newOp;
+ }
+
+ @Override
+ public int getNewValueSize() {
+ return newValueSize;
+ }
+
+ @Override
+ public void setNewValueSize(int newValueSize) {
+ this.newValueSize = newValueSize;
+ }
+
+ @Override
+ public ITupleReference getNewValue() {
+ return newValue;
+ }
+
+ @Override
+ public void setNewValue(ITupleReference newValue) {
+ this.newValue = newValue;
+ this.fieldCnt = newValue.getFieldCount();
+ }
+
+ @Override
+ public byte getOldOp() {
+ return oldOp;
+ }
+
+ @Override
+ public void setOldOp(byte oldOp) {
+ this.oldOp = oldOp;
+ }
+
+ @Override
+ public int getOldValueSize() {
+ return oldValueSize;
+ }
+
+ @Override
+ public void setOldValueSize(int oldValueSize) {
+ this.oldValueSize = oldValueSize;
+ }
+
+ @Override
+ public ITupleReference getOldValue() {
+ return oldValue;
+ }
+
+ @Override
+ public void setOldValue(ITupleReference oldValue) {
+ this.oldValue = oldValue;
+ }
+
+ @Override
+ public long getChecksum() {
+ return checksum;
+ }
+
+ @Override
+ public void setChecksum(long checksum) {
+ this.checksum = checksum;
+ }
+
+ @Override
+ public long getLSN() {
+ return LSN;
+ }
+
+ @Override
+ public void setLSN(long LSN) {
+ this.LSN = LSN;
+ }
+
+ @Override
+ public int getPKValueSize() {
+ return PKValueSize;
+ }
+
+ @Override
+ public ITupleReference getPKValue() {
+ return PKValue;
+ }
+
+ @Override
+ public void setPKFields(int[] primaryKeyFields) {
+ PKFields = primaryKeyFields;
+ PKFieldCnt = PKFields.length;
+ }
+
+ @Override
+ public void setPKValue(ITupleReference PKValue) {
+ this.PKValue = PKValue;
+ }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
deleted file mode 100644
index d46ade8..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-/**
- * An implementation of the @see ILogRecordHelper interface that provides API
- * for writing/reading of log header and checksum as well as validating log
- * record by checksum comparison. Every ILogManager implementation has an
- * associated ILogRecordHelper implementation.
- * == LogRecordFormat ==
- * [Header]
- * --------------------------- Header part1(17) : Both COMMIT and UPDATE log type have part1 fields
- * LogMagicNumber(4)
- * LogType(1)
- * JobId(4)
- * DatasetId(4) //stored in dataset_dataset in Metadata Node
- * PKHashValue(4)
- * --------------------------- Header part2(21) : Only UPDATE log type has part2 fields
- * PrevLSN(8) //only for UPDATE
- * ResourceId(8) //stored in .metadata of the corresponding index in NC node
- * ResourceMgrId(1)
- * LogRecordSize(4)
- * --------------------------- COMMIT doesn't have Body fields.
- * [Body] The Body size is given through the parameter reusableLogContentObjectLength
- * TupleFieldCount(4)
- * NewOp(1)
- * NewValueLength(4)
- * NewValue(NewValueLength)
- * OldOp(1)
- * OldValueLength(4)
- * OldValue(OldValueLength)
- * --------------------------- Both COMMIT and UPDATE have tail fields.
- * [Tail]
- * Checksum(8)
- */
-public class LogRecordHelper implements ILogRecordHelper {
-
- private final int LOG_CHECKSUM_SIZE = 8;
- private final int LOG_HEADER_PART1_SIZE = 17;
- private final int LOG_HEADER_PART2_SIZE = 21;
- private final int COMMIT_LOG_SIZE = LOG_HEADER_PART1_SIZE + LOG_CHECKSUM_SIZE;
-
- private final int MAGIC_NO_POS = 0;
- private final int LOG_TYPE_POS = 4;
- private final int JOB_ID_POS = 5;
- private final int DATASET_ID_POS = 9;
- private final int PK_HASH_VALUE_POS = 13;
- private final int PREV_LSN_POS = 17;
- private final int RESOURCE_ID_POS = 25;
- private final int RESOURCE_MGR_ID_POS = 33;
- private final int LOG_RECORD_SIZE_POS = 34;
-
- private ILogManager logManager;
-
- public LogRecordHelper(ILogManager logManager) {
- this.logManager = logManager;
- }
-
- @Override
- public byte getLogType(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().getByte(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS);
- }
-
- @Override
- public int getJobId(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + JOB_ID_POS);
- }
-
- @Override
- public int getDatasetId(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + DATASET_ID_POS);
- }
-
- @Override
- public int getPKHashValue(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + PK_HASH_VALUE_POS);
- }
-
- @Override
- public PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator) {
- long prevLsnValue = (logicalLogLocator.getBuffer())
- .readLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS);
- PhysicalLogLocator previousLogLocator = new PhysicalLogLocator(prevLsnValue, logManager);
- return previousLogLocator;
- }
-
- @Override
- public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator) {
- long prevLsnValue = (logicalLogLocator.getBuffer())
- .readLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS);
- if (prevLsnValue == -1) {
- return false;
- }
- physicalLogLocator.setLsn(prevLsnValue);
- return true;
- }
-
- @Override
- public long getResourceId(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getBuffer().readLong(logicalLogLocator.getMemoryOffset() + RESOURCE_ID_POS);
- }
-
- @Override
- public byte getResourceMgrId(LogicalLogLocator logicalLogLocater) {
- return logicalLogLocater.getBuffer().getByte(logicalLogLocater.getMemoryOffset() + RESOURCE_MGR_ID_POS);
- }
-
- @Override
- public int getLogContentSize(LogicalLogLocator logicalLogLocater) {
- if (getLogType(logicalLogLocater) == LogType.COMMIT || getLogType(logicalLogLocater) == LogType.ENTITY_COMMIT) {
- return 0;
- } else {
- return logicalLogLocater.getBuffer().readInt(logicalLogLocater.getMemoryOffset() + LOG_RECORD_SIZE_POS);
- }
- }
-
- @Override
- public long getLogChecksum(LogicalLogLocator logicalLogLocator) {
- return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset()
- + getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator))
- - LOG_CHECKSUM_SIZE);
- }
-
- @Override
- public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getMemoryOffset() + getLogHeaderSize(getLogType(logicalLogLocator));
- }
-
- @Override
- public int getLogContentEndPos(LogicalLogLocator logicalLogLocator) {
- return logicalLogLocator.getMemoryOffset()
- + getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator))
- - LOG_CHECKSUM_SIZE;
- }
-
- @Override
- public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator) {
- StringBuilder builder = new StringBuilder();
- byte logType = new Byte(getLogType(logicalLogLocator));
- String logTypeDisplay = null;
- switch (logType) {
- case LogType.COMMIT:
- logTypeDisplay = "COMMIT";
- break;
- case LogType.UPDATE:
- logTypeDisplay = "UPDATE";
- break;
- case LogType.ENTITY_COMMIT:
- logTypeDisplay = "ENTITY_COMMIT";
- break;
- }
- builder.append(" LSN : ").append(logicalLogLocator.getLsn());
- builder.append(" Log Type : ").append(logTypeDisplay);
- builder.append(" Job Id : ").append(getJobId(logicalLogLocator));
- builder.append(" Dataset Id : ").append(getDatasetId(logicalLogLocator));
- builder.append(" PK Hash Value : ").append(getPKHashValue(logicalLogLocator));
- if (logType == LogType.UPDATE) {
- builder.append(" PrevLSN : ").append(getPrevLSN(logicalLogLocator).getLsn());
- builder.append(" Resource Id : ").append(getResourceId(logicalLogLocator));
- builder.append(" ResourceMgr Id : ").append(getResourceMgrId(logicalLogLocator));
- builder.append(" Log Record Size : ").append(
- getLogRecordSize(logType, getLogContentSize(logicalLogLocator)));
- }
- return builder.toString();
- }
-
- @Override
- public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
- int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
- int logRecordSize) {
-
- /* magic no */
- (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + MAGIC_NO_POS,
- LogManagerProperties.LOG_MAGIC_NUMBER);
-
- /* log type */
- (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS, logType);
-
- /* jobId */
- (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + JOB_ID_POS, context.getJobId()
- .getId());
-
- /* datasetId */
- (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + DATASET_ID_POS, datasetId);
-
- /* PK hash value */
- (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + PK_HASH_VALUE_POS, PKHashValue);
-
- if (logType == LogType.UPDATE) {
- /* prevLSN */
- (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS,
- prevLogicalLogLocator);
-
- /* resourceId */
- (logicalLogLocator.getBuffer())
- .writeLong(logicalLogLocator.getMemoryOffset() + RESOURCE_ID_POS, resourceId);
-
- /* resourceMgr id */
- (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + RESOURCE_MGR_ID_POS,
- resourceMgrId);
-
- /* log record size */
- (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + LOG_RECORD_SIZE_POS,
- logRecordSize);
-
- }
- }
-
- @Override
- public boolean validateLogRecord(LogicalLogLocator logicalLogLocator) {
- int logLength = this.getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator));
- long expectedChecksum = DataUtil.getChecksum(logicalLogLocator.getBuffer(),
- logicalLogLocator.getMemoryOffset(), logLength - LOG_CHECKSUM_SIZE);
- long actualChecksum = getLogChecksum(logicalLogLocator);
- return expectedChecksum == actualChecksum;
- }
-
- /**
- * @param logType
- * @param logBodySize
- * @return
- */
- @Override
- public int getLogRecordSize(byte logType, int logBodySize) {
- if (logType == LogType.UPDATE) {
- return LOG_HEADER_PART1_SIZE + LOG_HEADER_PART2_SIZE + LOG_CHECKSUM_SIZE + logBodySize;
- } else {
- return COMMIT_LOG_SIZE;
- }
- }
-
- @Override
- public int getLogHeaderSize(byte logType) {
- if (logType == LogType.UPDATE) {
- return LOG_HEADER_PART1_SIZE + LOG_HEADER_PART2_SIZE;
- } else {
- return LOG_HEADER_PART1_SIZE;
- }
- }
-
- @Override
- public int getLogChecksumSize() {
- return LOG_CHECKSUM_SIZE;
- }
-
- public int getCommitLogSize() {
- return COMMIT_LOG_SIZE;
- }
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
index 99c0fef..f9e9304 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
@@ -17,7 +17,29 @@
public class LogType {
public static final byte UPDATE = 0;
- public static final byte COMMIT = 1;
+ public static final byte JOB_COMMIT = 1;
public static final byte ENTITY_COMMIT = 2;
+ public static final byte ABORT = 3;
+ private static final String STRING_UPDATE = "UPDATE";
+ private static final String STRING_JOB_COMMIT = "JOB_COMMIT";
+ private static final String STRING_ENTITY_COMMIT = "ENTITY_COMMIT";
+ private static final String STRING_ABORT = "ABORT";
+ private static final String STRING_INVALID_LOG_TYPE = "INVALID_LOG_TYPE";
+
+
+ public static String toString(byte logType) {
+ switch (logType) {
+ case LogType.UPDATE:
+ return STRING_UPDATE;
+ case LogType.JOB_COMMIT:
+ return STRING_JOB_COMMIT;
+ case LogType.ENTITY_COMMIT:
+ return STRING_ENTITY_COMMIT;
+ case LogType.ABORT:
+ return STRING_ABORT;
+ default:
+ return STRING_INVALID_LOG_TYPE;
+ }
+ }
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java
deleted file mode 100644
index d63ce1c..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.logging;
-
-import java.nio.ByteBuffer;
-
-/**
- * Represents a memory-only buffer that is not associated with and physical
- * file.
- */
-public class MemBasedBuffer extends Buffer {
-
- public MemBasedBuffer(byte[] content) {
- super(ByteBuffer.wrap(content));
- }
-
- public MemBasedBuffer(ByteBuffer buffer) {
- super(buffer);
- }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java
new file mode 100644
index 0000000..d45b209
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class PrimaryKeyTupleReference implements ITupleReference {
+ private byte[] fieldData;
+ private int start;
+ private int length;
+
+ public void reset(byte[] fieldData, int start, int length) {
+ this.fieldData = fieldData;
+ this.start = start;
+ this.length = length;
+ }
+
+ @Override
+ public int getFieldCount() {
+ return 1;
+ }
+
+ @Override
+ public byte[] getFieldData(int fIdx) {
+ return fieldData;
+ }
+
+ @Override
+ public int getFieldStart(int fIdx) {
+ return start;
+ }
+
+ @Override
+ public int getFieldLength(int fIdx) {
+ return length;
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
index 110ddee..f2cb22a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
@@ -20,24 +20,24 @@
private static final long serialVersionUID = 1L;
- private final long checkpointLSN;
- private final long minMCTFirstLSN;
+ private final long checkpointLsn;
+ private final long minMCTFirstLsn;
private final int maxJobId;
private final long timeStamp;
- public CheckpointObject(long checkpointLSN, long minMCTFirstLSN, int maxJobId, long timeStamp) {
- this.checkpointLSN = checkpointLSN;
- this.minMCTFirstLSN = minMCTFirstLSN;
+ public CheckpointObject(long checkpointLsn, long minMCTFirstLsn, int maxJobId, long timeStamp) {
+ this.checkpointLsn = checkpointLsn;
+ this.minMCTFirstLsn = minMCTFirstLsn;
this.maxJobId = maxJobId;
this.timeStamp = timeStamp;
}
- public long getCheckpointLSN() {
- return checkpointLSN;
+ public long getCheckpointLsn() {
+ return checkpointLsn;
}
- public long getMinMCTFirstLSN() {
- return minMCTFirstLSN;
+ public long getMinMCTFirstLsn() {
+ return minMCTFirstLsn;
}
public int getMaxJobId() {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
index a558969..dca14d8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
@@ -16,9 +16,10 @@
import java.util.List;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
@@ -56,7 +57,7 @@
try {
recoveryMgr.checkpoint(false);
lastMinMCTFirstLSN = currentMinMCTFirstLSN;
- } catch (ACIDException e) {
+ } catch (ACIDException | HyracksDataException e) {
throw new Error("failed to checkpoint", e);
}
}
@@ -69,7 +70,8 @@
long firstLSN;
if (openIndexList.size() > 0) {
for (IIndex index : openIndexList) {
- firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
+ firstLSN = ((AbstractLSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback())
+ .getFirstLSN();
minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
}
} else {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
index ad1db1f..00abc86 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -27,6 +27,7 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
@@ -37,17 +38,12 @@
import java.util.logging.Logger;
import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
@@ -60,23 +56,14 @@
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeImmutableComponent;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexImmutableComponent;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeImmutableComponent;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
import edu.uci.ics.hyracks.storage.common.file.LocalResource;
@@ -91,7 +78,9 @@
public static final boolean IS_DEBUG_MODE = false;//true
private static final Logger LOGGER = Logger.getLogger(RecoveryManager.class.getName());
private final TransactionSubsystem txnSubsystem;
+ private final LogManager logMgr;
private final int checkpointHistory;
+ private final long SHARP_CHECKPOINT_LSN = -1;
/**
* A file at a known location that contains the LSN of the last log record
@@ -100,8 +89,9 @@
private static final String CHECKPOINT_FILENAME_PREFIX = "checkpoint_";
private SystemState state;
- public RecoveryManager(TransactionSubsystem TransactionProvider) throws ACIDException {
- this.txnSubsystem = TransactionProvider;
+ public RecoveryManager(TransactionSubsystem txnSubsystem) throws ACIDException {
+ this.txnSubsystem = txnSubsystem;
+ this.logMgr = (LogManager) txnSubsystem.getLogManager();
this.checkpointHistory = this.txnSubsystem.getTransactionProperties().getCheckpointHistory();
}
@@ -129,12 +119,20 @@
return state;
}
- //#. if minMCTFirstLSN is equal to -1 &&
- // checkpointLSN in the checkpoint file is equal to the lastLSN in the log file,
- // then return healthy state. Otherwise, return corrupted.
- LogManager logMgr = (LogManager) txnSubsystem.getLogManager();
- if (checkpointObject.getMinMCTFirstLSN() == -1
- && checkpointObject.getCheckpointLSN() == logMgr.getCurrentLsn().get()) {
+ long readableSmallestLSN = logMgr.getReadableSmallestLSN();
+ if (logMgr.getAppendLSN() == readableSmallestLSN) {
+ if (checkpointObject.getMinMCTFirstLsn() != SHARP_CHECKPOINT_LSN) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("[Warning] ---------------------------------------------------");
+ LOGGER.info("[Warning] Some(or all) of transaction log files are lost.");
+ LOGGER.info("[Warning] ---------------------------------------------------");
+ //No choice but continuing when the log files are lost.
+ }
+ }
+ state = SystemState.HEALTHY;
+ return state;
+ } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN()
+ && checkpointObject.getMinMCTFirstLsn() == SHARP_CHECKPOINT_LSN) {
state = SystemState.HEALTHY;
return state;
} else {
@@ -146,151 +144,132 @@
public void startRecovery(boolean synchronous) throws IOException, ACIDException {
int updateLogCount = 0;
- int commitLogCount = 0;
+ int entityCommitLogCount = 0;
+ int jobCommitLogCount = 0;
int redoCount = 0;
+ int abortLogCount = 0;
+ int jobId = -1;
state = SystemState.RECOVERING;
- ILogManager logManager = txnSubsystem.getLogManager();
- ILogRecordHelper logRecordHelper = logManager.getLogRecordHelper();
-
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("[RecoveryMgr] starting recovery ...");
}
- //winnerTxnTable is used to add pairs, <committed TxnId, the most recent commit LSN of the TxnId>
- Map<TxnId, Long> winnerTxnTable = new HashMap<TxnId, Long>();
- TxnId tempKeyTxnId = new TxnId(-1, -1, -1);
- byte logType;
+ Set<Integer> winnerJobSet = new HashSet<Integer>();
+ Map<Integer, Set<TxnId>> jobId2WinnerEntitiesMap = new HashMap<Integer, Set<TxnId>>();
+ //winnerEntity is used to add pairs, <committed TxnId, the most recent commit Lsn of the TxnId>
+ Set<TxnId> winnerEntitySet = null;
+ TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
+ TxnId winnerEntity = null;
//#. read checkpoint file and set lowWaterMark where anaylsis and redo start
+ long readableSmallestLSN = logMgr.getReadableSmallestLSN();
CheckpointObject checkpointObject = readCheckpoint();
- long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLSN();
- if (lowWaterMarkLSN == -1) {
- lowWaterMarkLSN = 0;
+ long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLsn();
+ if (lowWaterMarkLSN < readableSmallestLSN) {
+ lowWaterMarkLSN = readableSmallestLSN;
}
int maxJobId = checkpointObject.getMaxJobId();
- int currentJobId;
//-------------------------------------------------------------------------
// [ analysis phase ]
- // - collect all committed LSN
- // - if there are duplicate commits for the same TxnId,
- // keep only the mostRecentCommitLSN among the duplicates.
+ // - collect all committed Lsn
//-------------------------------------------------------------------------
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("[RecoveryMgr] in analysis phase");
}
- //#. set log cursor to the lowWaterMarkLSN
- ILogCursor logCursor = logManager.readLog(new PhysicalLogLocator(lowWaterMarkLSN, logManager),
- new ILogFilter() {
- public boolean accept(IBuffer logs, long startOffset, int endOffset) {
- return true;
- }
- });
- LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
-
- //#. collect all committed txn's pairs,<TxnId, LSN>
- while (logCursor.next(currentLogLocator)) {
-
- if (LogManager.IS_DEBUG_MODE) {
- System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+ //#. set log reader to the lowWaterMarkLsn
+ ILogReader logReader = logMgr.getLogReader(true);
+ logReader.initializeScan(lowWaterMarkLSN);
+ ILogRecord logRecord = logReader.next();
+ while (logRecord != null) {
+ if (IS_DEBUG_MODE) {
+ LOGGER.info(logRecord.getLogRecordForDisplay());
}
-
- logType = logRecordHelper.getLogType(currentLogLocator);
-
//update max jobId
- currentJobId = logRecordHelper.getJobId(currentLogLocator);
- if (currentJobId > maxJobId) {
- maxJobId = currentJobId;
+ if (logRecord.getJobId() > maxJobId) {
+ maxJobId = logRecord.getJobId();
}
-
- TxnId commitTxnId = null;
- switch (logType) {
+ switch (logRecord.getLogType()) {
case LogType.UPDATE:
- if (IS_DEBUG_MODE) {
- updateLogCount++;
- }
+ updateLogCount++;
break;
-
- case LogType.COMMIT:
+ case LogType.JOB_COMMIT:
+ winnerJobSet.add(Integer.valueOf(logRecord.getJobId()));
+ jobId2WinnerEntitiesMap.remove(Integer.valueOf(logRecord.getJobId()));
+ jobCommitLogCount++;
+ break;
case LogType.ENTITY_COMMIT:
- commitTxnId = new TxnId(logRecordHelper.getJobId(currentLogLocator),
- logRecordHelper.getDatasetId(currentLogLocator),
- logRecordHelper.getPKHashValue(currentLogLocator));
- winnerTxnTable.put(commitTxnId, currentLogLocator.getLsn());
- if (IS_DEBUG_MODE) {
- commitLogCount++;
+ jobId = logRecord.getJobId();
+ winnerEntity = new TxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+ logRecord.getPKValue(), logRecord.getPKValueSize(), true);
+ if (!jobId2WinnerEntitiesMap.containsKey(Integer.valueOf(jobId))) {
+ winnerEntitySet = new HashSet<TxnId>();
+ jobId2WinnerEntitiesMap.put(Integer.valueOf(jobId), winnerEntitySet);
+ } else {
+ winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
}
+ winnerEntitySet.add(winnerEntity);
+ entityCommitLogCount++;
break;
-
+ case LogType.ABORT:
+ abortLogCount++;
+ break;
default:
- throw new ACIDException("Unsupported LogType: " + logType);
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
}
+ logRecord = logReader.next();
}
//-------------------------------------------------------------------------
// [ redo phase ]
// - redo if
- // 1) The TxnId is committed --> gurantee durability
- // &&
- // 2) the currentLSN > maxDiskLastLSN of the index --> guarantee idempotance
+ // 1) The TxnId is committed && --> guarantee durability
+ // 2) lsn > maxDiskLastLsn of the index --> guarantee idempotence
//-------------------------------------------------------------------------
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("[RecoveryMgr] in redo phase");
}
- //#. set log cursor to the lowWaterMarkLSN again.
- logCursor = logManager.readLog(new PhysicalLogLocator(lowWaterMarkLSN, logManager), new ILogFilter() {
- public boolean accept(IBuffer logs, long startOffset, int endOffset) {
- return true;
- }
- });
- currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
-
long resourceId;
- byte resourceMgrId;
- long maxDiskLastLSN;
- long currentLSN = -1;
- int resourceType;
+ long maxDiskLastLsn;
+ long LSN = -1;
ILSMIndex index = null;
LocalResource localResource = null;
ILocalResourceMetadata localResourceMetadata = null;
Map<Long, Long> resourceId2MaxLSNMap = new HashMap<Long, Long>();
- List<ILSMComponent> immutableDiskIndexList = null;
- TxnId jobLevelTxnId = new TxnId(-1, -1, -1);
- boolean foundWinnerTxn;
+ boolean foundWinner = false;
//#. get indexLifeCycleManager
IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
IIndexLifecycleManager indexLifecycleManager = appRuntimeContext.getIndexLifecycleManager();
ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
- //#. redo
- while (logCursor.next(currentLogLocator)) {
- foundWinnerTxn = false;
- if (LogManager.IS_DEBUG_MODE) {
- System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+ //#. set log reader to the lowWaterMarkLsn again.
+ logReader.initializeScan(lowWaterMarkLSN);
+ logRecord = logReader.next();
+ while (logRecord != null) {
+ if (IS_DEBUG_MODE) {
+ LOGGER.info(logRecord.getLogRecordForDisplay());
}
-
- logType = logRecordHelper.getLogType(currentLogLocator);
-
- switch (logType) {
+ LSN = logRecord.getLSN();
+ jobId = logRecord.getJobId();
+ foundWinner = false;
+ switch (logRecord.getLogType()) {
case LogType.UPDATE:
- tempKeyTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator),
- logRecordHelper.getDatasetId(currentLogLocator),
- logRecordHelper.getPKHashValue(currentLogLocator));
- jobLevelTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator), -1, -1);
- if (winnerTxnTable.containsKey(tempKeyTxnId)) {
- currentLSN = winnerTxnTable.get(tempKeyTxnId);
- foundWinnerTxn = true;
- } else if (winnerTxnTable.containsKey(jobLevelTxnId)) {
- currentLSN = winnerTxnTable.get(jobLevelTxnId);
- foundWinnerTxn = true;
+ if (winnerJobSet.contains(Integer.valueOf(jobId))) {
+ foundWinner = true;
+ } else if (jobId2WinnerEntitiesMap.containsKey(Integer.valueOf(jobId))) {
+ winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
+ tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+ logRecord.getPKValue(), logRecord.getPKValueSize());
+ if (winnerEntitySet.contains(tempKeyTxnId)) {
+ foundWinner = true;
+ }
}
-
- if (foundWinnerTxn) {
- resourceId = logRecordHelper.getResourceId(currentLogLocator);
+ if (foundWinner) {
+ resourceId = logRecord.getResourceId();
localResource = localResourceRepository.getResourceById(resourceId);
//get index instance from IndexLifeCycleManager
@@ -313,6 +292,7 @@
* log record.
*******************************************************************/
if (localResource == null) {
+ logRecord = logReader.next();
continue;
}
/*******************************************************************/
@@ -325,74 +305,33 @@
indexLifecycleManager.open(resourceId);
//#. get maxDiskLastLSN
- resourceType = localResource.getResourceType();
- immutableDiskIndexList = index.getImmutableComponents();
-
- maxDiskLastLSN = -1;
- switch (resourceType) {
-
- case ResourceType.LSM_BTREE:
- for (ILSMComponent c : immutableDiskIndexList) {
- BTree btree = ((LSMBTreeImmutableComponent) c).getBTree();
- maxDiskLastLSN = Math.max(getTreeIndexLSN(btree), maxDiskLastLSN);
- }
- break;
-
- case ResourceType.LSM_RTREE:
- for (ILSMComponent c : immutableDiskIndexList) {
- RTree rtree = ((LSMRTreeImmutableComponent) c).getRTree();
- maxDiskLastLSN = Math.max(getTreeIndexLSN(rtree), maxDiskLastLSN);
- }
- break;
-
- case ResourceType.LSM_INVERTED_INDEX:
- for (ILSMComponent c : immutableDiskIndexList) {
- BTree delKeyBtree = ((LSMInvertedIndexImmutableComponent) c)
- .getDeletedKeysBTree();
- maxDiskLastLSN = Math.max(getTreeIndexLSN(delKeyBtree), maxDiskLastLSN);
- }
- break;
-
- default:
- throw new ACIDException("Unsupported resouce type");
- }
+ ILSMIndex lsmIndex = (ILSMIndex) index;
+ maxDiskLastLsn = ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
+ .getComponentLSN(lsmIndex.getImmutableComponents());
//#. set resourceId and maxDiskLastLSN to the map
- resourceId2MaxLSNMap.put(resourceId, maxDiskLastLSN);
+ resourceId2MaxLSNMap.put(Long.valueOf(resourceId), Long.valueOf(maxDiskLastLsn));
} else {
- maxDiskLastLSN = resourceId2MaxLSNMap.get(resourceId);
+ maxDiskLastLsn = resourceId2MaxLSNMap.get(Long.valueOf(resourceId));
}
- if (currentLSN > maxDiskLastLSN) {
- resourceMgrId = logRecordHelper.getResourceMgrId(currentLogLocator);
-
- // look up the repository to get the resource manager
- // register resourceMgr if it is not registered.
- IResourceManager resourceMgr = txnSubsystem.getTransactionalResourceRepository()
- .getTransactionalResourceMgr(resourceMgrId);
- if (resourceMgr == null) {
- resourceMgr = new IndexResourceManager(resourceMgrId, txnSubsystem);
- txnSubsystem.getTransactionalResourceRepository().registerTransactionalResourceManager(
- resourceMgrId, resourceMgr);
- }
-
- //redo finally.
- resourceMgr.redo(logRecordHelper, currentLogLocator);
- if (IS_DEBUG_MODE) {
- redoCount++;
- }
+ if (LSN > maxDiskLastLsn) {
+ redo(logRecord);
+ redoCount++;
}
}
break;
- case LogType.COMMIT:
+ case LogType.JOB_COMMIT:
case LogType.ENTITY_COMMIT:
+ case LogType.ABORT:
//do nothing
break;
default:
- throw new ACIDException("Unsupported LogType: " + logType);
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
}
+ logRecord = logReader.next();
}
//close all indexes
@@ -401,42 +340,23 @@
indexLifecycleManager.close(r);
}
+ logReader.close();
+
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("[RecoveryMgr] recovery is completed.");
- }
- if (IS_DEBUG_MODE) {
- System.out.println("[RecoveryMgr] Count: Update/Commit/Redo = " + updateLogCount + "/" + commitLogCount
- + "/" + redoCount);
- }
- }
-
- //TODO
- //This function came from the AbstractLSMIOOperationCallback class.
- //We'd better factor out this function into a component of reading/writing the local metadata of indexes.
- private long getTreeIndexLSN(ITreeIndex treeIndex) throws HyracksDataException {
- int fileId = treeIndex.getFileId();
- IBufferCache bufferCache = treeIndex.getBufferCache();
- ITreeIndexMetaDataFrame metadataFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame();
- int metadataPageId = treeIndex.getFreePageManager().getFirstMetadataPage();
- ICachedPage metadataPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metadataPageId), false);
- metadataPage.acquireReadLatch();
- try {
- metadataFrame.setPage(metadataPage);
- return metadataFrame.getLSN();
- } finally {
- metadataPage.releaseReadLatch();
- bufferCache.unpin(metadataPage);
+ LOGGER.info("[RecoveryMgr's recovery log count] update/entityCommit/jobCommit/abort/redo = " + updateLogCount + "/"
+ + entityCommitLogCount + "/" + jobCommitLogCount + "/" + abortLogCount + "/" + redoCount);
}
}
@Override
- public synchronized void checkpoint(boolean isSharpCheckpoint) throws ACIDException {
+ public synchronized void checkpoint(boolean isSharpCheckpoint) throws ACIDException, HyracksDataException {
+ long minMCTFirstLSN;
if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting sharp checkpoint ... ");
}
- LogManager logMgr = (LogManager) txnSubsystem.getLogManager();
TransactionManager txnMgr = (TransactionManager) txnSubsystem.getTransactionManager();
String logDir = logMgr.getLogManagerProperties().getLogDir();
@@ -447,17 +367,20 @@
IIndexLifecycleManager indexLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
.getIndexLifecycleManager();
List<IIndex> openIndexList = indexLifecycleManager.getOpenIndexes();
- List<BlockingIOOperationCallbackWrapper> callbackList = new LinkedList<BlockingIOOperationCallbackWrapper>();
//#. flush all in-memory components if it is the sharp checkpoint
if (isSharpCheckpoint) {
+ ///////////////////////////////////////////////
+ //TODO : change the code inside the if statement into indexLifeCycleManager.flushAllDatasets()
+ //indexLifeCycleManager.flushAllDatasets();
+ ///////////////////////////////////////////////
+ List<BlockingIOOperationCallbackWrapper> callbackList = new LinkedList<BlockingIOOperationCallbackWrapper>();
for (IIndex index : openIndexList) {
ILSMIndex lsmIndex = (ILSMIndex) index;
ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(NoOpOperationCallback.INSTANCE,
NoOpOperationCallback.INSTANCE);
- BaseOperationTracker indexOpTracker = (BaseOperationTracker) lsmIndex.getOperationTracker();
BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
- indexOpTracker.getIOOperationCallback());
+ lsmIndex.getIOOperationCallback());
callbackList.add(cb);
try {
indexAccessor.scheduleFlush(cb);
@@ -473,21 +396,21 @@
throw new ACIDException(e);
}
}
- }
-
- //#. create and store the checkpointObject into the new checkpoint file
- long minMCTFirstLSN = Long.MAX_VALUE;
- long firstLSN;
- if (openIndexList.size() > 0) {
- for (IIndex index : openIndexList) {
- firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
- minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
- }
+ minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
} else {
- minMCTFirstLSN = -1;
+ long firstLSN;
+ minMCTFirstLSN = Long.MAX_VALUE;
+ if (openIndexList.size() > 0) {
+ for (IIndex index : openIndexList) {
+ firstLSN = ((AbstractLSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback())
+ .getFirstLSN();
+ minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
+ }
+ } else {
+ minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
+ }
}
-
- CheckpointObject checkpointObject = new CheckpointObject(logMgr.getCurrentLsn().get(), minMCTFirstLSN,
+ CheckpointObject checkpointObject = new CheckpointObject(logMgr.getAppendLSN(), minMCTFirstLSN,
txnMgr.getMaxJobId(), System.currentTimeMillis());
FileOutputStream fos = null;
@@ -585,7 +508,7 @@
}
private File[] getPreviousCheckpointFiles() {
- String logDir = txnSubsystem.getLogManager().getLogManagerProperties().getLogDir();
+ String logDir = ((LogManager) txnSubsystem.getLogManager()).getLogManagerProperties().getLogDir();
File parentDir = new File(logDir);
@@ -621,24 +544,25 @@
*/
@Override
public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException {
- ILogManager logManager = txnSubsystem.getLogManager();
- ILogRecordHelper logRecordHelper = logManager.getLogRecordHelper();
Map<TxnId, List<Long>> loserTxnTable = new HashMap<TxnId, List<Long>>();
- TxnId tempKeyTxnId = new TxnId(-1, -1, -1);
+ TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
int updateLogCount = 0;
- int commitLogCount = 0;
+ int entityCommitLogCount = 0;
+ int jobId = -1;
+ int abortedJobId = txnContext.getJobId().getId();
+ long currentLSN = -1;
+ TxnId loserEntity = null;
- // Obtain the first log record written by the Job
- PhysicalLogLocator firstLSNLogLocator = txnContext.getFirstLogLocator();
- PhysicalLogLocator lastLSNLogLocator = txnContext.getLastLogLocator();
+ // Obtain the first/last log record LSNs written by the Job
+ long firstLSN = txnContext.getFirstLSN();
+ long lastLSN = txnContext.getLastLSN();
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" rollbacking transaction log records from " + firstLSNLogLocator.getLsn() + " to "
- + lastLSNLogLocator.getLsn());
+ LOGGER.info(" rollbacking transaction log records from " + firstLSN + " to " + lastLSN);
}
// check if the transaction actually wrote some logs.
- if (firstLSNLogLocator.getLsn() == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
+ if (firstLSN == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(" no need to roll back as there were no operations by the transaction "
+ txnContext.getJobId());
@@ -646,146 +570,107 @@
return;
}
- // While reading log records from firstLSN to lastLSN, collect uncommitted txn's LSNs
- ILogCursor logCursor;
- try {
- logCursor = logManager.readLog(firstLSNLogLocator, new ILogFilter() {
- @Override
- public boolean accept(IBuffer buffer, long startOffset, int length) {
- return true;
- }
- });
- } catch (IOException e) {
- throw new ACIDException("Failed to create LogCursor with LSN:" + firstLSNLogLocator.getLsn(), e);
- }
-
- LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
- boolean valid;
- byte logType;
- List<Long> undoLSNSet = null;
-
+ // While reading log records from firstLsn to lastLsn, collect uncommitted txn's Lsns
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" collecting loser transaction's LSNs from " + firstLSNLogLocator.getLsn() + " to "
- + +lastLSNLogLocator.getLsn());
+ LOGGER.info(" collecting loser transaction's LSNs from " + firstLSN + " to " + lastLSN);
}
-
- while (currentLogLocator.getLsn() != lastLSNLogLocator.getLsn()) {
- try {
- valid = logCursor.next(currentLogLocator);
- } catch (IOException e) {
- throw new ACIDException("Failed to read log at LSN:" + currentLogLocator.getLsn(), e);
- }
- if (!valid) {
- if (currentLogLocator.getLsn() != lastLSNLogLocator.getLsn()) {
- throw new ACIDException("LastLSN mismatch: " + lastLSNLogLocator.getLsn() + " vs "
- + currentLogLocator.getLsn() + " during Rollback a transaction( " + txnContext.getJobId()
- + ")");
- } else {
- break;//End of Log File
+ List<Long> undoLSNSet = null;
+ ILogReader logReader = logMgr.getLogReader(false);
+ logReader.initializeScan(firstLSN);
+ ILogRecord logRecord = null;
+ while (currentLSN < lastLSN) {
+ logRecord = logReader.next();
+ if (logRecord == null) {
+ break;
+ } else {
+ if (IS_DEBUG_MODE) {
+ LOGGER.info(logRecord.getLogRecordForDisplay());
}
+ currentLSN = logRecord.getLSN();
}
-
- if (IS_DEBUG_MODE) {
- System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+ jobId = logRecord.getJobId();
+ if (jobId != abortedJobId) {
+ continue;
}
-
- tempKeyTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator),
- logRecordHelper.getDatasetId(currentLogLocator), logRecordHelper.getPKHashValue(currentLogLocator));
- logType = logRecordHelper.getLogType(currentLogLocator);
-
- switch (logType) {
+ tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(), logRecord.getPKValue(),
+ logRecord.getPKValueSize());
+ switch (logRecord.getLogType()) {
case LogType.UPDATE:
undoLSNSet = loserTxnTable.get(tempKeyTxnId);
if (undoLSNSet == null) {
- TxnId txnId = new TxnId(logRecordHelper.getJobId(currentLogLocator),
- logRecordHelper.getDatasetId(currentLogLocator),
- logRecordHelper.getPKHashValue(currentLogLocator));
+ loserEntity = new TxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+ logRecord.getPKValue(), logRecord.getPKValueSize(), true);
undoLSNSet = new LinkedList<Long>();
- loserTxnTable.put(txnId, undoLSNSet);
+ loserTxnTable.put(loserEntity, undoLSNSet);
}
- undoLSNSet.add(currentLogLocator.getLsn());
+ undoLSNSet.add(Long.valueOf(currentLSN));
+ updateLogCount++;
if (IS_DEBUG_MODE) {
- updateLogCount++;
- System.out.println("" + Thread.currentThread().getId() + "======> update["
- + currentLogLocator.getLsn() + "]:" + tempKeyTxnId);
+ LOGGER.info("" + Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
+ + tempKeyTxnId);
}
break;
- case LogType.COMMIT:
+ case LogType.JOB_COMMIT:
+ throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
+
case LogType.ENTITY_COMMIT:
- undoLSNSet = loserTxnTable.get(tempKeyTxnId);
- if (undoLSNSet != null) {
- loserTxnTable.remove(tempKeyTxnId);
+ undoLSNSet = loserTxnTable.remove(tempKeyTxnId);
+ if (undoLSNSet == null) {
+ undoLSNSet = loserTxnTable.remove(tempKeyTxnId);
}
+ entityCommitLogCount++;
if (IS_DEBUG_MODE) {
- commitLogCount++;
- System.out.println("" + Thread.currentThread().getId() + "======> commit["
- + currentLogLocator.getLsn() + "]" + tempKeyTxnId);
+ LOGGER.info("" + Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
+ + tempKeyTxnId);
}
break;
+ case LogType.ABORT:
+ //ignore
+ break;
+
default:
- throw new ACIDException("Unsupported LogType: " + logType);
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
}
}
+ if (currentLSN != lastLSN) {
+ throw new ACIDException("LastLSN mismatch: lastLSN(" + lastLSN + ") vs currentLSN(" + currentLSN
+ + ") during abort( " + txnContext.getJobId() + ")");
+ }
//undo loserTxn's effect
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(" undoing loser transaction's effect");
}
- TxnId txnId = null;
Iterator<Entry<TxnId, List<Long>>> iter = loserTxnTable.entrySet().iterator();
- byte resourceMgrId;
int undoCount = 0;
while (iter.hasNext()) {
//TODO
//Sort the lsns in order to undo in one pass.
Map.Entry<TxnId, List<Long>> loserTxn = (Map.Entry<TxnId, List<Long>>) iter.next();
- txnId = loserTxn.getKey();
-
undoLSNSet = loserTxn.getValue();
for (long undoLSN : undoLSNSet) {
- // here, all the log records are UPDATE type. So, we don't need to check the type again.
-
+ //here, all the log records are UPDATE type. So, we don't need to check the type again.
//read the corresponding log record to be undone.
- logManager.readLog(undoLSN, currentLogLocator);
-
+ logRecord = logReader.read(undoLSN);
+ if (logRecord == null) {
+ throw new ACIDException("IllegalState exception during abort( " + txnContext.getJobId() + ")");
+ }
if (IS_DEBUG_MODE) {
- System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+ LOGGER.info(logRecord.getLogRecordForDisplay());
}
-
- // extract the resource manager id from the log record.
- resourceMgrId = logRecordHelper.getResourceMgrId(currentLogLocator);
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine(logRecordHelper.getLogRecordForDisplay(currentLogLocator));
- }
-
- // look up the repository to get the resource manager
- IResourceManager resourceMgr = txnSubsystem.getTransactionalResourceRepository()
- .getTransactionalResourceMgr(resourceMgrId);
-
- // register resourceMgr if it is not registered.
- if (resourceMgr == null) {
- resourceMgr = new IndexResourceManager(resourceMgrId, txnSubsystem);
- txnSubsystem.getTransactionalResourceRepository().registerTransactionalResourceManager(
- resourceMgrId, resourceMgr);
- }
- resourceMgr.undo(logRecordHelper, currentLogLocator);
-
- if (IS_DEBUG_MODE) {
- undoCount++;
- }
+ undo(logRecord);
+ undoCount++;
}
}
-
+ logReader.close();
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(" undone loser transaction's effect");
- }
- if (IS_DEBUG_MODE) {
- System.out.println("UpdateLogCount/CommitLogCount/UndoCount:" + updateLogCount + "/" + commitLogCount + "/"
+ LOGGER.info("[RecoveryManager's rollback log count] update/entityCommit/undo:" + updateLogCount + "/" + entityCommitLogCount + "/"
+ undoCount);
}
}
@@ -799,39 +684,104 @@
public void stop(boolean dumpState, OutputStream os) {
//no op
}
+
+ private void undo(ILogRecord logRecord) {
+ try {
+ ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+ .getIndex(logRecord.getResourceId());
+ ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ if (logRecord.getResourceType() == ResourceType.LSM_BTREE) {
+ if (logRecord.getOldOp() != IndexOperation.NOOP.ordinal()) {
+ if (logRecord.getOldOp() == IndexOperation.DELETE.ordinal()) {
+ indexAccessor.forceDelete(logRecord.getOldValue());
+ } else {
+ indexAccessor.forceInsert(logRecord.getOldValue());
+ }
+ } else {
+ indexAccessor.forcePhysicalDelete(logRecord.getNewValue());
+ }
+ } else {
+ if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
+ indexAccessor.forceInsert(logRecord.getNewValue());
+ } else {
+ indexAccessor.forceDelete(logRecord.getNewValue());
+ }
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException("Failed to undo", e);
+ }
+ }
+
+ private void redo(ILogRecord logRecord) {
+ try {
+ ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+ .getIndex(logRecord.getResourceId());
+ ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
+ indexAccessor.forceInsert(logRecord.getNewValue());
+ } else if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
+ indexAccessor.forceDelete(logRecord.getNewValue());
+ } else {
+ throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
+ }
+ ((AbstractLSMIOOperationCallback) index.getIOOperationCallback()).updateLastLSN(logRecord.getLSN());
+ } catch (Exception e) {
+ throw new IllegalStateException("Failed to redo", e);
+ }
+ }
}
class TxnId {
+ public boolean isByteArrayPKValue;
public int jobId;
public int datasetId;
- public int pkHashVal;
+ public int pkHashValue;
+ public int pkSize;
+ public byte[] byteArrayPKValue;
+ public ITupleReference tupleReferencePKValue;
- public TxnId(int jobId, int datasetId, int pkHashVal) {
+ public TxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize,
+ boolean isByteArrayPKValue) {
this.jobId = jobId;
this.datasetId = datasetId;
- this.pkHashVal = pkHashVal;
+ this.pkHashValue = pkHashValue;
+ this.pkSize = pkSize;
+ this.isByteArrayPKValue = isByteArrayPKValue;
+ if (isByteArrayPKValue) {
+ this.byteArrayPKValue = new byte[pkSize];
+ readPKValueIntoByteArray(pkValue, pkSize, byteArrayPKValue);
+ } else {
+ this.tupleReferencePKValue = pkValue;
+ }
}
- public void setTxnId(int jobId, int datasetId, int pkHashVal) {
+ private void readPKValueIntoByteArray(ITupleReference pkValue, int pkSize, byte[] byteArrayPKValue) {
+ int readOffset = pkValue.getFieldStart(0);
+ byte[] readBuffer = pkValue.getFieldData(0);
+ for (int i = 0; i < pkSize; i++) {
+ byteArrayPKValue[i] = readBuffer[readOffset + i];
+ }
+ }
+
+ public void setTxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize) {
this.jobId = jobId;
this.datasetId = datasetId;
- this.pkHashVal = pkHashVal;
- }
-
- public void setTxnId(TxnId txnId) {
- this.jobId = txnId.jobId;
- this.datasetId = txnId.datasetId;
- this.pkHashVal = txnId.pkHashVal;
+ this.pkHashValue = pkHashValue;
+ this.tupleReferencePKValue = pkValue;
+ this.pkSize = pkSize;
+ isByteArrayPKValue = false;
}
@Override
public String toString() {
- return "[" + jobId + "," + datasetId + "," + pkHashVal + "]";
+ return "[" + jobId + "," + datasetId + "," + pkHashValue + "," + pkSize + "]";
}
@Override
public int hashCode() {
- return pkHashVal;
+ return pkHashValue;
}
@Override
@@ -843,7 +793,52 @@
return false;
}
TxnId txnId = (TxnId) o;
+ return (txnId.pkHashValue == pkHashValue && txnId.datasetId == datasetId && txnId.jobId == jobId
+ && pkSize == txnId.pkSize && isEqualTo(txnId));
+ }
- return (txnId.pkHashVal == pkHashVal && txnId.datasetId == datasetId && txnId.jobId == jobId);
+ private boolean isEqualTo(TxnId txnId) {
+ if (isByteArrayPKValue && txnId.isByteArrayPKValue) {
+ return isEqual(byteArrayPKValue, txnId.byteArrayPKValue, pkSize);
+ } else if (isByteArrayPKValue && (!txnId.isByteArrayPKValue)) {
+ return isEqual(byteArrayPKValue, txnId.tupleReferencePKValue, pkSize);
+ } else if ((!isByteArrayPKValue) && txnId.isByteArrayPKValue) {
+ return isEqual(txnId.byteArrayPKValue, tupleReferencePKValue, pkSize);
+ } else {
+ return isEqual(tupleReferencePKValue, txnId.tupleReferencePKValue, pkSize);
+ }
+ }
+
+ private boolean isEqual(byte[] a, byte[] b, int size) {
+ for (int i = 0; i < size; i++) {
+ if (a[i] != b[i]) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private boolean isEqual(byte[] a, ITupleReference b, int size) {
+ int readOffset = b.getFieldStart(0);
+ byte[] readBuffer = b.getFieldData(0);
+ for (int i = 0; i < size; i++) {
+ if (a[i] != readBuffer[readOffset + i]) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private boolean isEqual(ITupleReference a, ITupleReference b, int size) {
+ int aOffset = a.getFieldStart(0);
+ byte[] aBuffer = a.getFieldData(0);
+ int bOffset = b.getFieldStart(0);
+ byte[] bBuffer = b.getFieldData(0);
+ for (int i = 0; i < size; i++) {
+ if (aBuffer[aOffset + i] != bBuffer[bOffset + i]) {
+ return false;
+ }
+ }
+ return true;
}
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
deleted file mode 100644
index e64ec01..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.service.recovery;
-
-import edu.uci.ics.asterix.common.transactions.ITransactionManager;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-/**
- * Represents a bookkeeping data-structure that is populated duing the analysis
- * phase of recovery. It contains for each transaction, the transaction state,
- * the LSN corresponding to the last log record written by the transaction and
- * the log record written by the transaction that needs to be undone.
- */
-public class TransactionTableEntry {
-
- private long transactionId;
- private ITransactionManager.TransactionState transactionState;
- private PhysicalLogLocator lastLSN;
- private PhysicalLogLocator undoNextLSN;
-
- public TransactionTableEntry(long transactionId, ITransactionManager.TransactionState transactionState,
- PhysicalLogLocator lastLSN, PhysicalLogLocator undoNextLSN) {
- this.transactionId = transactionId;
- this.transactionState = transactionState;
- this.lastLSN = lastLSN;
- this.undoNextLSN = undoNextLSN;
- }
-
- public long getTransactionId() {
- return transactionId;
- }
-
- public void setTransactionId(long transactionId) {
- this.transactionId = transactionId;
- }
-
- public ITransactionManager.TransactionState getTransactionState() {
- return transactionState;
- }
-
- public void setTransactionState(ITransactionManager.TransactionState transactionState) {
- this.transactionState = transactionState;
- }
-
- public PhysicalLogLocator getLastLSN() {
- return lastLSN;
- }
-
- public void setLastLSN(PhysicalLogLocator lastLSN) {
- this.lastLSN = lastLSN;
- }
-
- public PhysicalLogLocator getUndoNextLSN() {
- return undoNextLSN;
- }
-
- public void setUndoNextLSN(PhysicalLogLocator undoNextLSN) {
- this.undoNextLSN = undoNextLSN;
- }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
index 97f2477..5d4806d 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
@@ -15,23 +15,13 @@
package edu.uci.ics.asterix.transaction.management.service.transaction;
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -39,38 +29,12 @@
import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
public class AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
- ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider, ILSMOperationTrackerProvider,
- ILSMIOOperationCallbackProvider {
+ ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider {
private static final long serialVersionUID = 1L;
- private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
- private final boolean isSecondary;
+ public static final AsterixRuntimeComponentsProvider RUNTIME_PROVIDER = new AsterixRuntimeComponentsProvider();
- public static final AsterixRuntimeComponentsProvider LSMBTREE_PRIMARY_PROVIDER = new AsterixRuntimeComponentsProvider(
- LSMBTreeIOOperationCallbackFactory.INSTANCE, false);
- public static final AsterixRuntimeComponentsProvider LSMBTREE_SECONDARY_PROVIDER = new AsterixRuntimeComponentsProvider(
- LSMBTreeIOOperationCallbackFactory.INSTANCE, true);
- public static final AsterixRuntimeComponentsProvider LSMRTREE_PROVIDER = new AsterixRuntimeComponentsProvider(
- LSMRTreeIOOperationCallbackFactory.INSTANCE, true);
- public static final AsterixRuntimeComponentsProvider LSMINVERTEDINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(
- LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, true);
- public static final AsterixRuntimeComponentsProvider NOINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(null,
- false);
-
- private AsterixRuntimeComponentsProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean isSecondary) {
- this.ioOpCallbackFactory = ioOpCallbackFactory;
- this.isSecondary = isSecondary;
- }
-
- @Override
- public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
- assert isSecondary;
- return new BaseOperationTracker(ioOpCallbackFactory);
- }
-
- @Override
- public ILSMIOOperationCallback getIOOperationCallback(ILSMIndex index) {
- return ((BaseOperationTracker) index.getOperationTracker()).getIOOperationCallback();
+ private AsterixRuntimeComponentsProvider() {
}
@Override
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
index 33522e3..4085964 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -15,202 +15,233 @@
package edu.uci.ics.asterix.transaction.management.service.transaction;
import java.io.Serializable;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.PrimaryIndexOperationTracker;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
-import edu.uci.ics.asterix.common.transactions.ICloseable;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallback;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
-/**
- * Represents a holder object that contains all information related to a
- * transaction. A TransactionContext instance can be used as a token and
- * provided to Transaction sub-systems (Log/Lock/Recovery/Transaction)Manager to
- * initiate an operation on the behalf of the transaction associated with the
- * context.
+/*
+ * An object of TransactionContext is created and accessed(read/written) by multiple threads which work for
+ * a single job identified by a jobId. Thus, the member variables in the object can be read/written
+ * concurrently. Please see each variable declaration to know which one is accessed concurrently and
+ * which one is not.
*/
public class TransactionContext implements ITransactionContext, Serializable {
- private static final long serialVersionUID = -6105616785783310111L;
- private TransactionSubsystem transactionSubsystem;
- private LogicalLogLocator firstLogLocator;//firstLSN of the Job
- private LogicalLogLocator lastLogLocator;//lastLSN of the Job
- private TransactionState txnState;
- private long startWaitTime;
- private int status;
- private Set<ICloseable> resources = new HashSet<ICloseable>();
- private TransactionType transactionType = TransactionType.READ;
- private JobId jobId;
- private boolean exlusiveJobLevelCommit;
+ private static final long serialVersionUID = -6105616785783310111L;
+ private TransactionSubsystem transactionSubsystem;
- // List of indexes on which operations were performed on behalf of this transaction.
- private final Set<ILSMIndex> indexes = new HashSet<ILSMIndex>();
+ // jobId is set once and read concurrently.
+ private final JobId jobId;
- // List of operation callbacks corresponding to the operand indexes. In particular, needed to track
- // the number of active operations contributed by this transaction.
- private final Set<AbstractOperationCallback> callbacks = new HashSet<AbstractOperationCallback>();
+ // There are no concurrent writers on both firstLSN and lastLSN
+ // since both values are updated by serialized log appenders.
+ // But readers and writers can be different threads,
+ // so both LSNs are atomic variables in order to be read and written
+ // atomically.
+ private AtomicLong firstLSN;
+ private AtomicLong lastLSN;
- public TransactionContext(JobId jobId, TransactionSubsystem transactionSubsystem) throws ACIDException {
- this.jobId = jobId;
- this.transactionSubsystem = transactionSubsystem;
- init();
- }
+ // txnState is read and written concurrently.
+ private AtomicInteger txnState;
- private void init() throws ACIDException {
- firstLogLocator = LogUtil.getDummyLogicalLogLocator(transactionSubsystem.getLogManager());
- lastLogLocator = LogUtil.getDummyLogicalLogLocator(transactionSubsystem.getLogManager());
- txnState = TransactionState.ACTIVE;
- startWaitTime = INVALID_TIME;
- status = ACTIVE_STATUS;
- }
+ // isTimeout is read and written under the lockMgr's tableLatch
+ // Thus, no other synchronization is required separately.
+ private boolean isTimeout;
- public void registerIndexAndCallback(ILSMIndex index, AbstractOperationCallback callback) {
- synchronized (indexes) {
- indexes.add(index);
- callbacks.add(callback);
- }
- }
+ // isWriteTxn can be set concurrently by multiple threads.
+ private AtomicBoolean isWriteTxn;
- public void updateLastLSNForIndexes(long lastLSN) {
- synchronized (indexes) {
- for (ILSMIndex index : indexes) {
- ((BaseOperationTracker) index.getOperationTracker()).updateLastLSN(lastLSN);
- }
- }
- }
+ // isMetadataTxn is accessed by a single thread since the metadata is not
+ // partitioned
+ private boolean isMetadataTxn;
- public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException {
- synchronized (indexes) {
- Set<BaseOperationTracker> opTrackers = new HashSet<BaseOperationTracker>();
- Iterator<ILSMIndex> indexIt = indexes.iterator();
- Iterator<AbstractOperationCallback> cbIt = callbacks.iterator();
- while (indexIt.hasNext()) {
- ILSMIndex index = indexIt.next();
- opTrackers.add((BaseOperationTracker) index.getOperationTracker());
- assert cbIt.hasNext();
- }
- Iterator<BaseOperationTracker> trackerIt = opTrackers.iterator();
- while (trackerIt.hasNext()) {
- IModificationOperationCallback modificationCallback = (IModificationOperationCallback) cbIt.next();
- BaseOperationTracker opTracker = (BaseOperationTracker) trackerIt.next();
- if (exlusiveJobLevelCommit) {
- opTracker.exclusiveJobCommitted();
- } else {
- opTracker.completeOperation(null, LSMOperationType.MODIFICATION, null, modificationCallback);
- }
- }
- }
- }
+ // indexMap is concurrently accessed by multiple threads,
+ // so those threads are synchronized on indexMap object itself
+ private Map<MutableLong, AbstractLSMIOOperationCallback> indexMap;
- @Override
- public int getActiveOperationCountOnIndexes() throws HyracksDataException {
- synchronized (indexes) {
- int count = 0;
- Iterator<AbstractOperationCallback> cbIt = callbacks.iterator();
- while (cbIt.hasNext()) {
- IModificationOperationCallback modificationCallback = (IModificationOperationCallback) cbIt.next();
- count += ((AbstractOperationCallback) modificationCallback).getLocalNumActiveOperations();
- }
- return count;
- }
- }
+ // TODO: fix ComponentLSNs' issues.
+ // primaryIndex, primaryIndexCallback, and primaryIndexOptracker will be
+ // modified accordingly
+ // when the issues of componentLSNs are fixed.
+ private ILSMIndex primaryIndex;
+ private PrimaryIndexModificationOperationCallback primaryIndexCallback;
+ private PrimaryIndexOperationTracker primaryIndexOpTracker;
- public void setTransactionType(TransactionType transactionType) {
- this.transactionType = transactionType;
- }
+ // The following three variables are used as temporary variables in order to
+ // avoid object creations.
+ // Those are used in synchronized methods.
+ private MutableLong tempResourceIdForRegister;
+ private MutableLong tempResourceIdForSetLSN;
+ private LogRecord logRecord;
- public TransactionType getTransactionType() {
- return transactionType;
- }
+ // TODO: implement transactionContext pool in order to avoid object
+ // creations.
+ // also, the pool can throttle the number of concurrent active jobs at every
+ // moment.
+ public TransactionContext(JobId jobId,
+ TransactionSubsystem transactionSubsystem) throws ACIDException {
+ this.jobId = jobId;
+ this.transactionSubsystem = transactionSubsystem;
+ firstLSN = new AtomicLong(-1);
+ lastLSN = new AtomicLong(-1);
+ txnState = new AtomicInteger(ITransactionManager.ACTIVE);
+ isTimeout = false;
+ isWriteTxn = new AtomicBoolean(false);
+ isMetadataTxn = false;
+ indexMap = new HashMap<MutableLong, AbstractLSMIOOperationCallback>();
+ primaryIndex = null;
+ tempResourceIdForRegister = new MutableLong();
+ tempResourceIdForSetLSN = new MutableLong();
+ logRecord = new LogRecord();
+ }
- public void addCloseableResource(ICloseable resource) {
- resources.add(resource);
- }
+ public void registerIndexAndCallback(long resourceId, ILSMIndex index,
+ AbstractOperationCallback callback, boolean isPrimaryIndex) {
+ synchronized (indexMap) {
+ if (isPrimaryIndex && primaryIndex == null) {
+ primaryIndex = index;
+ primaryIndexCallback = (PrimaryIndexModificationOperationCallback) callback;
+ primaryIndexOpTracker = (PrimaryIndexOperationTracker) index
+ .getOperationTracker();
+ }
+ tempResourceIdForRegister.set(resourceId);
+ if (!indexMap.containsKey(tempResourceIdForRegister)) {
+ indexMap.put(new MutableLong(resourceId),
+ ((AbstractLSMIOOperationCallback) index
+ .getIOOperationCallback()));
+ }
+ }
+ }
- public LogicalLogLocator getFirstLogLocator() {
- return firstLogLocator;
- }
+ // [Notice]
+ // This method is called sequentially by the LogAppender threads.
+ // However, the indexMap is concurrently read and modified through this
+ // method and registerIndexAndCallback()
+ @Override
+ public void setLastLSN(long resourceId, long LSN) {
+ synchronized (indexMap) {
+ firstLSN.compareAndSet(-1, LSN);
+ lastLSN.set(Math.max(lastLSN.get(), LSN));
+ if (resourceId != -1) {
+ // Non-update log's resourceId is -1.
+ tempResourceIdForSetLSN.set(resourceId);
+ AbstractLSMIOOperationCallback ioOpCallback = indexMap
+ .get(tempResourceIdForSetLSN);
+ ioOpCallback.updateLastLSN(LSN);
+ }
+ }
+ }
- public LogicalLogLocator getLastLogLocator() {
- return lastLogLocator;
- }
+ @Override
+ public void notifyOptracker(boolean isJobLevelCommit) {
+ try {
+ if (isJobLevelCommit && isMetadataTxn) {
+ primaryIndexOpTracker.exclusiveJobCommitted();
+ } else if (!isJobLevelCommit) {
+ primaryIndexOpTracker.completeOperation(null,
+ LSMOperationType.MODIFICATION, null,
+ primaryIndexCallback);
+ }
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ }
- public void setLastLSN(long lsn) {
- if (firstLogLocator.getLsn() == -1) {
- firstLogLocator.setLsn(lsn);
- }
- lastLogLocator.setLsn(lsn);
- }
+ public void setWriteTxn(boolean isWriteTxn) {
+ this.isWriteTxn.set(isWriteTxn);
+ }
- public JobId getJobId() {
- return jobId;
- }
+ public boolean isWriteTxn() {
+ return isWriteTxn.get();
+ }
- public void setStartWaitTime(long time) {
- this.startWaitTime = time;
- }
+ @Override
+ public long getFirstLSN() {
+ return firstLSN.get();
+ }
- public long getStartWaitTime() {
- return startWaitTime;
- }
+ @Override
+ public long getLastLSN() {
+ return lastLSN.get();
+ }
- public void setStatus(int status) {
- this.status = status;
- }
+ public JobId getJobId() {
+ return jobId;
+ }
- public int getStatus() {
- return status;
- }
+ public void setTimeout(boolean isTimeout) {
+ this.isTimeout = isTimeout;
+ }
- public void setTxnState(TransactionState txnState) {
- this.txnState = txnState;
- }
+ public boolean isTimeout() {
+ return isTimeout;
+ }
- public TransactionState getTxnState() {
- return txnState;
- }
+ public void setTxnState(int txnState) {
+ this.txnState.set(txnState);
+ }
- public void releaseResources() throws ACIDException {
- for (ICloseable closeable : resources) {
- closeable.close(this);
- }
- }
+ public int getTxnState() {
+ return txnState.get();
+ }
- @Override
- public int hashCode() {
- return jobId.getId();
- }
+ @Override
+ public int hashCode() {
+ return jobId.getId();
+ }
- @Override
- public boolean equals(Object o) {
- return (o == this);
- }
+ @Override
+ public boolean equals(Object o) {
+ return (o == this);
+ }
- @Override
- public void setExclusiveJobLevelCommit() {
- exlusiveJobLevelCommit = true;
- }
+ @Override
+ public void setMetadataTransaction(boolean isMetadataTxn) {
+ this.isMetadataTxn = isMetadataTxn;
+ }
- public String prettyPrint() {
- StringBuilder sb = new StringBuilder();
- sb.append("\n" + jobId + "\n");
- sb.append("transactionType: " + transactionType);
- sb.append("firstLogLocator: " + firstLogLocator.getLsn() + "\n");
- sb.append("lastLogLocator: " + lastLogLocator.getLsn() + "\n");
- sb.append("TransactionState: " + txnState + "\n");
- sb.append("startWaitTime: " + startWaitTime + "\n");
- sb.append("status: " + status + "\n");
- return sb.toString();
- }
+ @Override
+ public boolean isMetadataTransaction() {
+ return isMetadataTxn;
+ }
+
+ public String prettyPrint() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("\n" + jobId + "\n");
+ sb.append("isWriteTxn: " + isWriteTxn + "\n");
+ sb.append("firstLSN: " + firstLSN.get() + "\n");
+ sb.append("lastLSN: " + lastLSN.get() + "\n");
+ sb.append("TransactionState: " + txnState + "\n");
+ sb.append("isTimeout: " + isTimeout + "\n");
+ return sb.toString();
+ }
+
+ public LogRecord getLogRecord() {
+ return logRecord;
+ }
+
+ public void cleanupForAbort() {
+ if (primaryIndexOpTracker != null) {
+ primaryIndexOpTracker
+ .cleanupNumActiveOperationsForAbortedJob(primaryIndexCallback);
+ }
+ }
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
index 54ec8a1..07fc152 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
@@ -16,9 +16,9 @@
import java.io.IOException;
import java.io.OutputStream;
-import java.util.HashMap;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -28,8 +28,7 @@
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
/**
@@ -40,115 +39,83 @@
public static final boolean IS_DEBUG_MODE = false;//true
private static final Logger LOGGER = Logger.getLogger(TransactionManager.class.getName());
- private final TransactionSubsystem transactionProvider;
- private Map<JobId, ITransactionContext> transactionContextRepository = new HashMap<JobId, ITransactionContext>();
+ private final TransactionSubsystem txnSubsystem;
+ private Map<JobId, ITransactionContext> transactionContextRepository = new ConcurrentHashMap<JobId, ITransactionContext>();
private AtomicInteger maxJobId = new AtomicInteger(0);
public TransactionManager(TransactionSubsystem provider) {
- this.transactionProvider = provider;
+ this.txnSubsystem = provider;
}
@Override
- public void abortTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
- throws ACIDException {
- synchronized (txnContext) {
- if (txnContext.getTxnState().equals(TransactionState.ABORTED)) {
- return;
+ public void abortTransaction(ITransactionContext txnCtx, DatasetId datasetId, int PKHashVal) throws ACIDException {
+ if (txnCtx.getTxnState() != ITransactionManager.ABORTED) {
+ txnCtx.setTxnState(ITransactionManager.ABORTED);
+ }
+ try {
+ if (txnCtx.isWriteTxn()) {
+ LogRecord logRecord = ((TransactionContext) txnCtx).getLogRecord();
+ logRecord.formJobTerminateLogRecord(txnCtx, false);
+ txnSubsystem.getLogManager().log(logRecord);
+ txnSubsystem.getRecoveryManager().rollbackTransaction(txnCtx);
}
-
- try {
- transactionProvider.getRecoveryManager().rollbackTransaction(txnContext);
- } catch (Exception ae) {
- String msg = "Could not complete rollback! System is in an inconsistent state";
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe(msg);
- }
- ae.printStackTrace();
- throw new Error(msg);
- } finally {
- txnContext.releaseResources();
- transactionProvider.getLockManager().releaseLocks(txnContext);
- transactionContextRepository.remove(txnContext.getJobId());
- txnContext.setTxnState(TransactionState.ABORTED);
+ } catch (Exception ae) {
+ String msg = "Could not complete rollback! System is in an inconsistent state";
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe(msg);
}
+ ae.printStackTrace();
+ throw new ACIDException(msg, ae);
+ } finally {
+ ((TransactionContext) txnCtx).cleanupForAbort();
+ txnSubsystem.getLockManager().releaseLocks(txnCtx);
+ transactionContextRepository.remove(txnCtx.getJobId());
}
}
@Override
public ITransactionContext beginTransaction(JobId jobId) throws ACIDException {
- setMaxJobId(jobId.getId());
- ITransactionContext txnContext = new TransactionContext(jobId, transactionProvider);
- synchronized (this) {
- transactionContextRepository.put(jobId, txnContext);
- }
- return txnContext;
+ return getTransactionContext(jobId, true);
}
@Override
- public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException {
+ public ITransactionContext getTransactionContext(JobId jobId, boolean createIfNotExist) throws ACIDException {
setMaxJobId(jobId.getId());
- synchronized (transactionContextRepository) {
-
- ITransactionContext context = transactionContextRepository.get(jobId);
- if (context == null) {
- context = transactionContextRepository.get(jobId);
- context = new TransactionContext(jobId, transactionProvider);
- transactionContextRepository.put(jobId, context);
- }
- return context;
- }
- }
-
- @Override
- public void commitTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
- throws ACIDException {
- synchronized (txnContext) {
- if ((txnContext.getTxnState().equals(TransactionState.COMMITTED))) {
- return;
- }
-
- //There is either job-level commit or entity-level commit.
- //The job-level commit will have -1 value both for datasetId and PKHashVal.
-
- //for entity-level commit
- if (PKHashVal != -1) {
- boolean countIsZero = transactionProvider.getLockManager().unlock(datasetId, PKHashVal, txnContext,
- true);
- if (!countIsZero) {
- // Lock count != 0 for a particular entity implies that the entity has been locked
- // more than once (probably due to a hash collision in our current model).
- // It is safe to decrease the active transaction count on indexes since,
- // by virtue of the counter not being zero, there is another transaction
- // that has increased the transaction count. Thus, decreasing it will not
- // allow the data to be flushed (yet). The flush will occur when the log page
- // flush thread decides to decrease the count for the last time.
- try {
- //decrease the transaction reference count on index
- txnContext.decreaseActiveTransactionCountOnIndexes();
- } catch (HyracksDataException e) {
- throw new ACIDException("failed to complete index operation", e);
+ ITransactionContext txnCtx = transactionContextRepository.get(jobId);
+ if (txnCtx == null) {
+ if (createIfNotExist) {
+ synchronized (this) {
+ txnCtx = transactionContextRepository.get(jobId);
+ if (txnCtx == null) {
+ txnCtx = new TransactionContext(jobId, txnSubsystem);
+ transactionContextRepository.put(jobId, txnCtx);
}
}
- return;
+ } else {
+ throw new ACIDException("TransactionContext of " + jobId + " doesn't exist.");
}
+ }
+ return txnCtx;
+ }
- //for job-level commit
- try {
- if (txnContext.getTransactionType().equals(ITransactionContext.TransactionType.READ_WRITE)) {
- transactionProvider.getLogManager().log(LogType.COMMIT, txnContext, -1, -1, -1, (byte) 0, 0, null,
- null, txnContext.getLastLogLocator());
- }
- } catch (ACIDException ae) {
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe(" caused exception in commit !" + txnContext.getJobId());
- }
- throw ae;
- } finally {
- txnContext.releaseResources();
- transactionProvider.getLockManager().releaseLocks(txnContext); // release
- transactionContextRepository.remove(txnContext.getJobId());
- txnContext.setTxnState(TransactionState.COMMITTED);
+ @Override
+ public void commitTransaction(ITransactionContext txnCtx, DatasetId datasetId, int PKHashVal) throws ACIDException {
+ //Only job-level commits call this method.
+ try {
+ if (txnCtx.isWriteTxn()) {
+ LogRecord logRecord = ((TransactionContext) txnCtx).getLogRecord();
+ logRecord.formJobTerminateLogRecord(txnCtx, true);
+ txnSubsystem.getLogManager().log(logRecord);
}
+ } catch (Exception ae) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe(" caused exception in commit !" + txnCtx.getJobId());
+ }
+ throw ae;
+ } finally {
+ txnSubsystem.getLockManager().releaseLocks(txnCtx); // release
+ transactionContextRepository.remove(txnCtx.getJobId());
+ txnCtx.setTxnState(ITransactionManager.COMMITTED);
}
}
@@ -164,11 +131,14 @@
@Override
public TransactionSubsystem getTransactionProvider() {
- return transactionProvider;
+ return txnSubsystem;
}
public void setMaxJobId(int jobId) {
- maxJobId.set(Math.max(maxJobId.get(), jobId));
+ int maxId = maxJobId.get();
+ if (jobId > maxId) {
+ maxJobId.compareAndSet(maxId, jobId);
+ }
}
public int getMaxJobId() {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
index b76cf11..aceeb82 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
@@ -22,9 +22,7 @@
import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLoggerRepository;
import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
import edu.uci.ics.asterix.transaction.management.service.recovery.CheckpointThread;
import edu.uci.ics.asterix.transaction.management.service.recovery.RecoveryManager;
@@ -39,8 +37,6 @@
private final ILockManager lockManager;
private final ITransactionManager transactionManager;
private final IRecoveryManager recoveryManager;
- private final TransactionalResourceManagerRepository resourceRepository;
- private final IndexLoggerRepository loggerRepository;
private final IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider;
private final CheckpointThread checkpointThread;
private final AsterixTransactionProperties txnProperties;
@@ -50,11 +46,9 @@
this.id = id;
this.txnProperties = txnProperties;
this.transactionManager = new TransactionManager(this);
- this.logManager = new LogManager(this);
this.lockManager = new LockManager(this);
+ this.logManager = new LogManager(this);
this.recoveryManager = new RecoveryManager(this);
- this.loggerRepository = new IndexLoggerRepository(this);
- this.resourceRepository = new TransactionalResourceManagerRepository();
this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
if (asterixAppRuntimeContextProvider != null) {
this.checkpointThread = new CheckpointThread(recoveryManager,
@@ -81,14 +75,6 @@
return recoveryManager;
}
- public TransactionalResourceManagerRepository getTransactionalResourceRepository() {
- return resourceRepository;
- }
-
- public IndexLoggerRepository getTreeLoggerRepository() {
- return loggerRepository;
- }
-
public IAsterixAppRuntimeContextProvider getAsterixAppRuntimeContextProvider() {
return asterixAppRuntimeContextProvider;
}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
deleted file mode 100644
index 06379b4..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.logging;
-
-import java.util.Random;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-
-public class BasicLogger implements ILogger {
-
- private static long averageContentCreationTime = 0;
- private static long count = 0;
-
- public void log(ITransactionContext context, LogicalLogLocator wMemLSN, int length,
- ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-
- byte[] logContent = getRandomBytes(length);
- try {
- long startTime2 = System.nanoTime();
-
- IBuffer buffer = (IBuffer) (wMemLSN.getBuffer());
-
- /*
- * synchronized(buffer){ buffer.position(wMemLSN.getMemoryOffset());
- * buffer.put(logContent); }
- */
-
- byte[] logPageStorage = buffer.getArray();
- System.arraycopy(logContent, 0, logPageStorage, wMemLSN.getMemoryOffset(), logContent.length);
-
- /*
- * for(int i=0;i<logContent.length;i++){
- * ((IFileBasedBuffer)(wMemLSN.
- * getBuffer())).put(wMemLSN.getMemoryOffset() + i, logContent[i]);
- * }
- */
- long endTime2 = System.nanoTime();
- averageContentCreationTime = ((averageContentCreationTime * count) + (endTime2 - startTime2)) / (++count);
- } catch (Exception e) {
- throw new ACIDException("", e);
- }
- }
-
- public static long getAverageContentCreationTime() {
- return averageContentCreationTime;
- }
-
- public static long getNumLogs() {
- return averageContentCreationTime;
- }
-
- public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
- // TODO Auto-generated method stub
-
- }
-
- public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
- // TODO Auto-generated method stub
-
- }
-
- private static byte[] getRandomBytes(int size) {
- byte[] b = new byte[size];
- Random random = new Random();
- int num = random.nextInt(30);
- Integer number = (new Integer(num + 65));
- byte numByte = number.byteValue();
- for (int i = 0; i < size; i++) {
- b[i] = numByte;
- }
- return b;
- }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
deleted file mode 100644
index 8b9c970..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.logging;
-
-import edu.uci.ics.asterix.common.transactions.ILogger;
-
-public interface IResource {
-
- public byte[] getId();
-
- public ILogger getLogger();
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
deleted file mode 100644
index e4a8e45..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.logging.test;
-
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class RecoverySimulator {
-
- private static IRecoveryManager recoveryManager;
-
- public static void startRecovery() throws IOException, ACIDException {
- recoveryManager.startRecovery(true);
- }
-
- public static void main(String args[]) throws IOException, ACIDException, AsterixException {
- String id = "nc1";
- try {
- TransactionSubsystem factory = new TransactionSubsystem(id, null, new AsterixTransactionProperties(
- new AsterixPropertiesAccessor()));
- IRecoveryManager recoveryManager = factory.getRecoveryManager();
- recoveryManager.startRecovery(true);
- } catch (ACIDException acide) {
- acide.printStackTrace();
- throw acide;
- }
- }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
deleted file mode 100644
index 2eea43e..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.test;
-
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileLogger implements ILogger {
-
- IResource resource;
- String logRecordContent;
-
- public FileLogger(IResource resource) {
- this.resource = resource;
- }
-
- public int generateLogRecordContent(int currentValue, int finalValue) {
- StringBuilder builder = new StringBuilder();
- builder.append("" + currentValue + " " + finalValue);
- logRecordContent = new String(builder);
- return resource.getId().length + logRecordContent.length();
- }
-
- @Override
- public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
- // TODO Auto-generated method stub
-
- }
-
- @Override
- public void log(ITransactionContext context, final LogicalLogLocator memLSN, int logContentSize,
- ReusableLogContentObject reusableLogContentObject) throws ACIDException {
- byte[] buffer = memLSN.getBuffer().getArray();
- byte[] content = logRecordContent.getBytes();
- for (int i = 0; i < resource.getId().length; i++) {
- buffer[memLSN.getMemoryOffset() + i] = resource.getId()[i];
- }
- for (int i = 0; i < content.length; i++) {
- buffer[memLSN.getMemoryOffset() + resource.getId().length + i] = content[i];
- }
- }
-
- @Override
- public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
- // TODO Auto-generated method stub
-
- }
-
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
deleted file mode 100644
index 072738f..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.test;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileResource implements IResource {
-
- private byte[] resourceId = new byte[] { 1 };
- private ILogger logger;
-
- private File file;
- private int memCounter = 0;
- private int diskCounter = 0;
-
- public int getMemoryCounter() {
- return memCounter;
- }
-
- public int getDiskCounter() {
- return diskCounter;
- }
-
- public static enum CounterOperation {
- INCREMENT,
- DECREMENT,
- };
-
- public FileResource(String fileDir, String fileName) throws IOException {
- File dirFile = new File(fileDir);
- if (!dirFile.exists()) {
- FileUtil.createNewDirectory(fileDir);
- }
- file = new File(fileDir + "/" + fileName);
- if (!file.exists()) {
- FileUtil.createFileIfNotExists(file.getAbsolutePath());
- BufferedWriter writer = new BufferedWriter(new FileWriter(file));
- writer.write("0");
- writer.flush();
- } else {
- FileReader fileReader = new FileReader(file);
- BufferedReader bufferedReader = new BufferedReader(fileReader);
- String content = bufferedReader.readLine();
- diskCounter = Integer.parseInt(content);
- }
- logger = new FileLogger(this);
- }
-
- public synchronized void increment() {
- memCounter++;
- }
-
- public synchronized void decrement() {
- memCounter--;
- }
-
- public synchronized void setValue(int value) {
- memCounter = value;
- }
-
- public synchronized void sync() throws IOException {
- BufferedWriter writer = new BufferedWriter(new FileWriter(file));
- writer.write("" + memCounter);
- writer.flush();
- }
-
- public synchronized boolean checkIfValueInSync(int expectedValue) throws IOException {
- FileReader fileReader = new FileReader(file);
- BufferedReader bufferedReader = new BufferedReader(fileReader);
- String content = bufferedReader.readLine();
- return content.equals("" + expectedValue);
- }
-
- @Override
- public byte[] getId() {
- return resourceId;
- }
-
- @Override
- public ILogger getLogger() {
- return logger;
- }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
deleted file mode 100644
index 5a6e408..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileResourceManager implements IResourceManager {
-
- public static final byte id = 100;
- private Map<String, FileResource> transactionalResources = new HashMap<String, FileResource>();
-
- public void registerTransactionalResource(IResource resource) throws ACIDException {
- if (resource instanceof FileResource) {
- if (transactionalResources.get(new String(resource.getId())) == null) {
- transactionalResources.put(new String(resource.getId()), (FileResource) resource);
- }
- } else {
- throw new ACIDException(" invalid resource type :" + resource);
- }
- }
-
- @Override
- public byte getResourceManagerId() {
- return id;
- }
-
- @Override
- public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException {
- LogRecordInfo logRecordInfo = new LogRecordInfo(logRecordHelper, memLSN);
- FileResource fileManager = transactionalResources.get(logRecordInfo.getResourceId());
- if (fileManager == null) {
- throw new ACIDException(" Un-registered transactional resource :" + logRecordInfo.getResourceId());
- }
- fileManager.setValue(logRecordInfo.getBeforeValue());
- }
-
- @Override
- public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException {
- LogRecordInfo logRecordInfo = new LogRecordInfo(logRecordHelper, memLSN);
- FileResource fileManager = transactionalResources.get(new String(logRecordInfo.getResourceId()));
- if (fileManager == null) {
- throw new ACIDException(" Un-registered transactional resource :" + logRecordInfo.getResourceId());
- }
- fileManager.setValue(logRecordInfo.getAfterValue());
- }
-
-}
-
-class LogRecordInfo {
-
- byte[] resourceId;
- int beforeValue;
- int afterValue;
-
- public LogRecordInfo(ILogRecordHelper logParser, LogicalLogLocator memLSN) throws ACIDException {
- int logContentBeginPos = logParser.getLogContentBeginPos(memLSN);
- int logContentEndPos = logParser.getLogContentEndPos(memLSN);
- byte[] bufferContent = memLSN.getBuffer().getArray();
- resourceId = new byte[] { bufferContent[logContentBeginPos] };
- String content = new String(bufferContent, logContentBeginPos + resourceId.length, logContentEndPos
- - (logContentBeginPos + resourceId.length));
- beforeValue = Integer.parseInt(content.split(" ")[0]);
- afterValue = Integer.parseInt(content.split(" ")[1]);
- }
-
- public byte[] getResourceId() {
- return resourceId;
- }
-
- public void setResourceId(byte[] resourceId) {
- this.resourceId = resourceId;
- }
-
- public int getAfterValue() {
- return afterValue;
- }
-
- public void setFinalValue(int afterValue) {
- this.afterValue = afterValue;
- }
-
- public int getBeforeValue() {
- return beforeValue;
- }
-
- public void setBeforeValue(int beforeValue) {
- this.beforeValue = beforeValue;
- }
-
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
deleted file mode 100644
index 5195658..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.transaction.management.test;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-public class TransactionRecoverySimulator {
-
- public boolean recoverSystem() throws ACIDException {
- return true;
- }
-
- public static void main(String args[]) {
- TransactionRecoverySimulator recoverySimulator = new TransactionRecoverySimulator();
- }
-}