├── .gitignore ├── .travis.yml ├── LICENSE ├── NOTICE ├── README.markdown ├── bin ├── install.sh ├── recompile.sh ├── squall_cluster.sh ├── squall_local.sh └── storm_env.sh ├── build.sbt ├── contrib ├── bheaven-0.0.3.jar ├── storm-graphite-0.2.4-SNAPSHOT-all.jar └── ujmp-complete-0.2.5.jar ├── logo └── logo.jpg ├── project ├── build.properties └── plugins.sbt ├── squall-core └── src │ ├── main │ └── java │ │ └── ch │ │ └── epfl │ │ └── data │ │ └── squall │ │ ├── api │ │ └── sql │ │ │ ├── estimators │ │ │ ├── ConfigSelectivityEstimator.java │ │ │ ├── HardCodedSelectivities.java │ │ │ ├── JSQLTypeConverter.java │ │ │ ├── SelectivityEstimator.java │ │ │ └── SelingerSelectivityEstimator.java │ │ │ ├── main │ │ │ └── ParserMain.java │ │ │ ├── optimizers │ │ │ ├── CompGen.java │ │ │ ├── Optimizer.java │ │ │ ├── index │ │ │ │ ├── CompLevel.java │ │ │ │ ├── EarlyProjection.java │ │ │ │ ├── IndexCompGen.java │ │ │ │ ├── IndexRuleOptimizer.java │ │ │ │ ├── IndexSimpleOptimizer.java │ │ │ │ ├── IndexTranslator.java │ │ │ │ ├── LevelAssigner.java │ │ │ │ ├── RuleParallelismAssigner.java │ │ │ │ └── TableSelector.java │ │ │ └── name │ │ │ │ ├── CostEstimator.java │ │ │ │ ├── CostParallelismAssigner.java │ │ │ │ ├── CostParams.java │ │ │ │ ├── NameCompGen.java │ │ │ │ ├── NameCompGenFactory.java │ │ │ │ ├── NameCostOptimizer.java │ │ │ │ ├── NameManualOptimizer.java │ │ │ │ ├── NameManualParOptimizer.java │ │ │ │ ├── NameRuleOptimizer.java │ │ │ │ ├── NameTranslator.java │ │ │ │ ├── ProjGlobalCollect.java │ │ │ │ ├── ProjSchemaCreator.java │ │ │ │ └── manual_batching │ │ │ │ ├── ClusterConstants.java │ │ │ │ ├── ManualBatchingCompGenFactory.java │ │ │ │ ├── ManualBatchingOptimizer.java │ │ │ │ └── ManualBatchingParallelismAssigner.java │ │ │ ├── schema │ │ │ ├── ColumnNameType.java │ │ │ ├── Schema.java │ │ │ └── parser │ │ │ │ ├── ParseException.java │ │ │ │ ├── SchemaParser.java │ │ │ │ ├── SchemaParserConstants.java │ │ │ │ ├── SchemaParserTokenManager.java │ │ │ │ ├── SimpleCharStream.java │ │ │ │ ├── Token.java │ │ │ │ └── TokenMgrError.java │ │ │ ├── util │ │ │ ├── HierarchyExtractor.java │ │ │ ├── ImproperParallelismException.java │ │ │ ├── JoinTablesExprs.java │ │ │ ├── NotFromMyBranchException.java │ │ │ ├── ParserUtil.java │ │ │ ├── TableAliasName.java │ │ │ └── TupleSchema.java │ │ │ └── visitors │ │ │ ├── jsql │ │ │ ├── AndVisitor.java │ │ │ ├── ColumnCollectVisitor.java │ │ │ ├── JoinTablesExprsVisitor.java │ │ │ ├── MaxSubExpressionsVisitor.java │ │ │ ├── PrintVisitor.java │ │ │ └── SQLVisitor.java │ │ │ └── squall │ │ │ ├── ColumnRefCollectVisitor.java │ │ │ ├── IndexJoinHashVisitor.java │ │ │ ├── IndexSelectItemsVisitor.java │ │ │ ├── IndexWhereVisitor.java │ │ │ ├── NameJoinHashVisitor.java │ │ │ ├── NameProjectVisitor.java │ │ │ ├── NameSelectItemsVisitor.java │ │ │ ├── NameWhereVisitor.java │ │ │ ├── VECollectPredVisitor.java │ │ │ └── VECollectVisitor.java │ │ ├── components │ │ ├── AbstractComponent.java │ │ ├── AbstractJoinerComponent.java │ │ ├── Component.java │ │ ├── ComponentProperties.java │ │ ├── DataSourceComponent.java │ │ ├── EquiJoinComponent.java │ │ ├── JoinerComponent.java │ │ ├── OperatorComponent.java │ │ ├── hyper_cube │ │ │ ├── HyperCubeJoinComponent.java │ │ │ └── HyperCubeJoinComponentFactory.java │ │ └── theta │ │ │ ├── AdaptiveThetaJoinComponent.java │ │ │ ├── ThetaJoinComponent.java │ │ │ └── ThetaJoinComponentFactory.java │ │ ├── connectors │ │ └── hdfs │ │ │ └── HDFSmaterializer.java │ │ ├── ewh │ │ ├── algorithms │ │ │ ├── BSPAlgorithm.java │ │ │ ├── DenseMonotonicWeightPrecomputation.java │ │ │ ├── DenseWeightPrecomputation.java │ │ │ ├── InputOutputShallowCoarsener.java │ │ │ ├── InputShallowCoarsener.java │ │ │ ├── MCMCRandomWalkAlgorithm.java │ │ │ ├── OkcanAlgorithm.java │ │ │ ├── OkcanCandidateCoarsener.java │ │ │ ├── OkcanCandidateInputAlgorithm.java │ │ │ ├── OkcanCandidateOutputAlgorithm.java │ │ │ ├── OkcanCoarsener.java │ │ │ ├── OkcanExactCoarsener.java │ │ │ ├── OkcanExactInputAlgorithm.java │ │ │ ├── OkcanExactOutputAlgorithm.java │ │ │ ├── OutputShallowCoarsener.java │ │ │ ├── PWeightPrecomputation.java │ │ │ ├── ShallowCoarsener.java │ │ │ ├── TilingAlgorithm.java │ │ │ ├── WeightPrecomputation.java │ │ │ └── optimality │ │ │ │ ├── MaxAvgOptimality.java │ │ │ │ ├── OptimalityMetricInterface.java │ │ │ │ └── WeightFunction.java │ │ ├── components │ │ │ ├── DummyComponent.java │ │ │ ├── EWHSampleMatrixComponent.java │ │ │ ├── EquiDepthHistogramComponent.java │ │ │ └── OkcanSampleMatrixComponent.java │ │ ├── data_structures │ │ │ ├── BooleanRegions.java │ │ │ ├── ExtremePositions.java │ │ │ ├── FixedSizePriorityQueue.java │ │ │ ├── FrequencyPosition.java │ │ │ ├── JoinMatrix.java │ │ │ ├── KeyPriorityProbability.java │ │ │ ├── KeyRegion.java │ │ │ ├── ListAdapter.java │ │ │ ├── ListJavaGeneric.java │ │ │ ├── ListTIntAdapter.java │ │ │ ├── ListTLongAdapter.java │ │ │ ├── MatrixIntInt.java │ │ │ ├── NumOfBuckets.java │ │ │ ├── Point.java │ │ │ ├── Region.java │ │ │ ├── SimpleMatrix.java │ │ │ ├── SparseMatrixUJMP.java │ │ │ ├── UJMPAdapterByteMatrix.java │ │ │ └── UJMPAdapterIntMatrix.java │ │ ├── examples │ │ │ ├── EWHSampleMatrixPlan.java │ │ │ ├── OkcanSampleMatrixPlan.java │ │ │ ├── ThetaEWHBandJPS.java │ │ │ ├── ThetaEWHBandLineitemSelfOrderkeyJoin.java │ │ │ ├── ThetaEWHBandOrdersCustkeyCustkeyJoin.java │ │ │ ├── ThetaEWHBandOrdersOrderkeyCustkeyJoin.java │ │ │ ├── ThetaEWHBandPeer.java │ │ │ ├── ThetaEWHCustomerJoin.java │ │ │ ├── ThetaEWHEquiLineitemOrders.java │ │ │ ├── ThetaEWHEquiOrdersCustkeyCustkeyJoin.java │ │ │ ├── ThetaEWHLineitemSelfOutputDominatedJoin.java │ │ │ ├── ThetaEWHOrdersScaleJoin.java │ │ │ └── ThetaEWHPartSuppJoin.java │ │ ├── main │ │ │ ├── PullStatisticCollector.java │ │ │ └── PushStatisticCollector.java │ │ ├── operators │ │ │ └── SampleAsideAndForwardOperator.java │ │ ├── storm_components │ │ │ ├── D2CombinerBolt.java │ │ │ ├── DummyBolt.java │ │ │ ├── EWHSampleMatrixBolt.java │ │ │ ├── EquiDepthHistogramBolt.java │ │ │ ├── OkcanSampleMatrixBolt.java │ │ │ ├── S1ReservoirGenerator.java │ │ │ ├── S1ReservoirMerge.java │ │ │ └── stream_grouping │ │ │ │ ├── RangeFilteredMulticastStreamGrouping.java │ │ │ │ └── RangeMulticastStreamGrouping.java │ │ ├── utilities │ │ │ ├── OverweightedException.java │ │ │ ├── TooSmallMaxWeightException.java │ │ │ └── TroveIntArrayTest.java │ │ └── visualize │ │ │ ├── UJMPVisualizer.java │ │ │ └── VisualizerInterface.java │ │ ├── expressions │ │ ├── Addition.java │ │ ├── ColumnReference.java │ │ ├── ConcatIntDouble.java │ │ ├── ConcatIntString.java │ │ ├── DateDiff.java │ │ ├── DateSum.java │ │ ├── Division.java │ │ ├── DoubleToInt.java │ │ ├── IntegerYearFromDate.java │ │ ├── LongPhone.java │ │ ├── Multiplication.java │ │ ├── StringConcatenate.java │ │ ├── Subtraction.java │ │ ├── ValueExpression.java │ │ └── ValueSpecification.java │ │ ├── main │ │ └── Main.java │ │ ├── operators │ │ ├── AggregateAvgOperator.java │ │ ├── AggregateCountOperator.java │ │ ├── AggregateOperator.java │ │ ├── AggregateStream.java │ │ ├── AggregateSumOperator.java │ │ ├── ChainOperator.java │ │ ├── CustomSampleOperatorReachGraph.java │ │ ├── DistinctOperator.java │ │ ├── MultiAggregateOperator.java │ │ ├── OneToOneOperator.java │ │ ├── Operator.java │ │ ├── PrintOperator.java │ │ ├── ProjectOperator.java │ │ ├── RedisOperator.java │ │ ├── SampleOperator.java │ │ ├── SelectOperator.java │ │ └── StoreOperator.java │ │ ├── predicates │ │ ├── AndPredicate.java │ │ ├── BetweenPredicate.java │ │ ├── ComparisonPredicate.java │ │ ├── LikePredicate.java │ │ ├── OrPredicate.java │ │ ├── Predicate.java │ │ └── booleanPrimitive.java │ │ ├── query_plans │ │ ├── QueryBuilder.java │ │ ├── QueryPlan.java │ │ └── ThetaQueryPlansParameters.java │ │ ├── storage │ │ ├── AggregationStore.java │ │ ├── BasicStore.java │ │ ├── BitVector.java │ │ ├── KeyValueStore.java │ │ ├── TupleStorage.java │ │ ├── ValueStore.java │ │ ├── WindowAggregationStore.java │ │ ├── WindowKeyValueStore.java │ │ └── indexes │ │ │ ├── BalancedBinaryTreeIndex.java │ │ │ ├── BplusTreeIndex.java │ │ │ ├── HashIndex.java │ │ │ └── Index.java │ │ ├── storm_components │ │ ├── SignaledDataSourceComponentInterface.java │ │ ├── StormBoltComponent.java │ │ ├── StormComponent.java │ │ ├── StormDataSource.java │ │ ├── StormDstJoin.java │ │ ├── StormDstTupleStorageJoin.java │ │ ├── StormEmitter.java │ │ ├── StormJoinerBoltComponent.java │ │ ├── StormOperator.java │ │ ├── StormSpoutComponent.java │ │ ├── StormSrcHarmonizer.java │ │ ├── StormSrcJoin.java │ │ ├── StormSrcStorage.java │ │ ├── SynchronizedStormDataSourceInterface.java │ │ ├── hash_hypercube │ │ │ └── HashHyperCubeGrouping.java │ │ ├── hybrid_hypercube │ │ │ └── HybridHyperCubeGrouping.java │ │ ├── hyper_cube │ │ │ ├── StormHyperCubeJoin.java │ │ │ ├── TradionalTwoWayJoin.java │ │ │ ├── TraditionalStormHyperCubeJoin.java │ │ │ └── stream_grouping │ │ │ │ └── HyperCubeGrouping.java │ │ ├── stream_grouping │ │ │ ├── BatchStreamGrouping.java │ │ │ ├── HashStreamGrouping.java │ │ │ └── ShuffleStreamGrouping.java │ │ ├── synchronization │ │ │ └── TopologyKiller.java │ │ └── theta │ │ │ ├── StormThetaJoin.java │ │ │ └── stream_grouping │ │ │ ├── ContentInsensitiveThetaJoinGrouping.java │ │ │ └── ContentSensitiveThetaJoinGrouping.java │ │ ├── thetajoin │ │ ├── adaptive │ │ │ ├── advisor │ │ │ │ ├── Action.java │ │ │ │ ├── Advisor.java │ │ │ │ ├── Discard.java │ │ │ │ ├── Maybe.java │ │ │ │ ├── Migration.java │ │ │ │ └── TheoreticalAdvisorNew.java │ │ │ ├── storm_component │ │ │ │ ├── Quadruple.java │ │ │ │ ├── ThetaJoinerAdaptiveAdvisedEpochs.java │ │ │ │ └── ThetaReshufflerAdvisedEpochs.java │ │ │ └── storm_matrix_mapping │ │ │ │ ├── ThetaDataMigrationJoinerToReshufflerMapping.java │ │ │ │ └── ThetaJoinAdaptiveMapping.java │ │ ├── matrix_assignment │ │ │ ├── ArrangementIterator.java │ │ │ ├── CellIterator.java │ │ │ ├── ContentInsensitiveMatrixAssignment.java │ │ │ ├── ContentSensitiveMatrixAssignment.java │ │ │ ├── CostModel.java │ │ │ ├── CubeNAssignmentBruteForce.java │ │ │ ├── CubeNAssignmentEqui.java │ │ │ ├── HashHyperCubeAssignment.java │ │ │ ├── HashHyperCubeAssignmentBruteForce.java │ │ │ ├── HybridHyperCubeAssignment.java │ │ │ ├── HybridHyperCubeAssignmentBruteForce.java │ │ │ ├── HyperCubeAssignerFactory.java │ │ │ ├── HyperCubeAssignment.java │ │ │ ├── MBucketIPartitioning.java │ │ │ ├── ManualHybridHyperCubeAssignment.java │ │ │ ├── MatrixAssignment.java │ │ │ ├── SetArrangementIterator.java │ │ │ ├── Utilities.java │ │ │ └── exp │ │ │ │ ├── HCEquiPartitionExp.java │ │ │ │ ├── HCPartitionExp.java │ │ │ │ └── TimeoutController.java │ │ └── predicate_analyser │ │ │ └── PredicateAnalyser.java │ │ ├── types │ │ ├── DateIntegerType.java │ │ ├── DateLongType.java │ │ ├── DateType.java │ │ ├── DoubleType.java │ │ ├── IntegerType.java │ │ ├── LongType.java │ │ ├── NumericType.java │ │ ├── StringType.java │ │ ├── SumCount.java │ │ ├── SumCountType.java │ │ └── Type.java │ │ ├── utilities │ │ ├── CustomReader.java │ │ ├── DeepCopy.java │ │ ├── FileReaderProvider.java │ │ ├── LocalMergeResults.java │ │ ├── MyUtilities.java │ │ ├── PartitioningScheme.java │ │ ├── PeriodicAggBatchSend.java │ │ ├── ReaderProvider.java │ │ ├── SerializableFileInputStream.java │ │ ├── SerializableHDFSFileInputStream.java │ │ ├── SquallContext.java │ │ ├── SquallSerializationDelegate.java │ │ ├── StormWrapper.java │ │ ├── SystemParameters.java │ │ ├── statistics │ │ │ └── StatisticsUtilities.java │ │ └── thetajoin │ │ │ └── dynamic │ │ │ ├── BufferedTuple.java │ │ │ ├── ThetaJoinUtilities.java │ │ │ └── ThetaState.java │ │ ├── visitors │ │ ├── OperatorVisitor.java │ │ ├── PredicateCreateIndexesVisitor.java │ │ ├── PredicateUpdateIndexesVisitor.java │ │ ├── PredicateVisitor.java │ │ └── ValueExpressionVisitor.java │ │ └── window_semantics │ │ └── WindowSemanticsManager.java │ └── test │ ├── resources │ └── logback-test.xml │ └── scala │ ├── HyracksTest.scala │ ├── RSTTest.scala │ ├── TPCHTest.scala │ ├── TestSuite.scala │ ├── ThetaTest.scala │ ├── dbtoaster │ └── DbtoasterTest.scala │ └── sql │ ├── SqlHyracksTest.scala │ ├── SqlTpch10Test.scala │ ├── SqlTpch12Test.scala │ ├── SqlTpch3Test.scala │ ├── SqlTpch4Test.scala │ ├── SqlTpch5Test.scala │ ├── SqlTpch6Test.scala │ ├── SqlTpch7Test.scala │ ├── SqlTpch8Test.scala │ └── SqlTpch9Test.scala ├── squall-examples └── squall-java-examples │ └── src │ └── ch │ └── epfl │ └── data │ └── squall │ └── examples │ └── imperative │ ├── debug │ ├── HyracksL1Plan.java │ ├── HyracksL1SelectDatePlan.java │ ├── HyracksL1SelectIntPlan.java │ ├── HyracksL3BatchPlan.java │ ├── HyracksL3Plan.java │ ├── TPCH10_CustomPlan.java │ ├── TPCH3L1Plan.java │ ├── TPCH3L23Plan.java │ ├── TPCH3L2Plan.java │ ├── TPCH5PlanAvg.java │ ├── TPCH5_CustomPlan.java │ ├── TPCH5_R_N_S_LPlan.java │ ├── TPCH7_CustomPlan.java │ ├── TPCH7_L_S_N1Plan.java │ ├── TPCH8_9_P_LPlan.java │ ├── TPCH9_CustomPlan.java │ ├── ThetaLineitemSelfJoinInputDominated2_32.java │ ├── ThetaLineitemSelfJoinInputDominated4_16.java │ ├── ThetaLineitemSelfJoinInputDominated8_8.java │ ├── ThetaTPCH5_R_N_S_LPlan.java │ ├── ThetaTPCH7_CustomPlan.java │ ├── ThetaTPCH7_L_S_N1Plan.java │ └── ThetaTPCH8_9_P_LPlan.java │ ├── shj │ ├── HyracksPlan.java │ ├── HyracksPreAggPlan.java │ ├── HyracksPredicatePlan.java │ ├── RSTPlan.java │ ├── SimpleAggregationPlan.java │ ├── TPCH10Plan.java │ ├── TPCH3Plan.java │ ├── TPCH3PredicatePlan.java │ ├── TPCH4Plan.java │ ├── TPCH5Plan.java │ ├── TPCH5PredicatePlan.java │ ├── TPCH7Plan.java │ ├── TPCH7PlanHDFSMaterializer.java │ ├── TPCH8Plan.java │ └── TPCH9Plan.java │ ├── theta │ ├── ThetaHyracksPlan.java │ ├── ThetaInputDominatedPlan.java │ ├── ThetaLineitemPricesSelfJoin.java │ ├── ThetaLineitemSelfJoin.java │ ├── ThetaLineitemSelfJoinInputDominated.java │ ├── ThetaMultipleJoinPlan.java │ ├── ThetaOrdersSelfJoin.java │ ├── ThetaOutputDominatedPlan.java │ ├── ThetaTPCH10Plan.java │ ├── ThetaTPCH3Plan.java │ ├── ThetaTPCH4Plan.java │ ├── ThetaTPCH5Plan.java │ ├── ThetaTPCH7Plan.java │ ├── ThetaTPCH8Plan.java │ └── ThetaTPCH9Plan.java │ └── traditional │ ├── TradionalHypercubeThetaHyracksPlan.java │ ├── TraditionalGoogleMostFailedMachine.java │ ├── TraditionalHashTPCH3Plan.java │ ├── TraditionalHashTPCH9PartialPlan.java │ ├── TraditionalHybridReachability.java │ ├── TraditionalHybridTPCH3Plan.java │ ├── TraditionalHybridTPCH9PartialPlan.java │ ├── TraditionalHybridUrlReachability.java │ ├── TraditionalHyracksPlan.java │ ├── TraditionalTPCH3Plan.java │ └── TraditionalTPCH5Plan.java ├── squall-functional ├── macros │ └── Macros.scala └── src │ └── main │ └── scala │ └── ch │ └── epfl │ └── data │ └── squall │ └── api │ └── scala │ ├── REPL.scala │ ├── SquallType.scala │ ├── Stream.scala │ ├── TPCHSchema.scala │ ├── operators │ ├── ScalaAggregateOperator.scala │ ├── ScalaAggregationStorage.scala │ ├── ScalaFlatMapOperator.scala │ ├── ScalaMapOperator.scala │ ├── ScalaWindowAggregationStore.scala │ └── predicates │ │ └── ScalaPredicate.scala │ └── queries │ ├── ScalaHyracksPlan.scala │ ├── ScalaTPCH3Plan.scala │ └── ScalaTPCH7Plan.scala ├── squall-signals └── src │ ├── examples │ └── ch │ │ └── epfl │ │ └── data │ │ └── squall │ │ └── examples │ │ └── imperative │ │ └── sync │ │ └── TestSync.java │ └── main │ └── java │ └── ch │ └── epfl │ └── data │ └── squall │ └── components │ └── signal_components │ ├── DistributionSignalSpout.java │ ├── HarmonizerSignalSpout.java │ ├── Histogram.java │ ├── SignalUtilities.java │ ├── SignaledDataSourceComponent.java │ ├── StormSynchronizedSpoutComponent.java │ ├── SynchronizedStormDataSource.java │ └── storm │ ├── AbstractSignalConnection.java │ ├── BaseSignalBolt.java │ ├── BaseSignalSpout.java │ ├── SignalClient.java │ ├── SignalListener.java │ ├── StandaloneSignalConnection.java │ └── StormSignalConnection.java └── test ├── data ├── google │ ├── README │ ├── job_events.csv │ ├── machine_attributes.csv │ ├── machine_events.csv │ ├── schema.csv │ ├── task_constraints.csv │ ├── task_events.csv │ └── task_usage.csv ├── jps │ └── 12K │ │ ├── generate_synthetic_jps.py │ │ ├── jps_1.tbl │ │ └── jps_2.tbl ├── link_graph │ └── 0.01G │ │ ├── sd-arc.txt │ │ └── sd-index.txt ├── pavlo_torrent │ ├── sample │ │ └── peersnapshot-01.tbl │ └── schema.txt ├── rst │ ├── 100 │ │ ├── r.dat │ │ ├── s.dat │ │ └── t.dat │ └── 10K │ │ ├── r.dat │ │ ├── s.dat │ │ └── t.dat └── tpch │ └── 0.01G │ ├── customer.tbl │ ├── lineitem.tbl │ ├── nation.tbl │ ├── orders.tbl │ ├── part.tbl │ ├── partsupp.tbl │ ├── region.tbl │ └── supplier.tbl ├── results ├── link_graph │ └── 0.01G │ │ └── reachability_3.result ├── rst │ └── 10K │ │ └── rst.result └── tpch │ ├── 0.01G │ ├── hyracks.result │ ├── theta_input_dominated.result │ ├── theta_lines_self_join.result │ ├── theta_lines_self_join_input_dominated.result │ ├── theta_lines_self_join_mat.result │ ├── theta_multiple_join.result │ ├── theta_output_dominated.result │ ├── theta_tpch5_R_N_S_L.result │ ├── theta_tpch7_L_S_N1.result │ ├── tpch10.result │ ├── tpch12.result │ ├── tpch17.result │ ├── tpch19.result │ ├── tpch3.result │ ├── tpch4.result │ ├── tpch5.result │ ├── tpch5avg.result │ ├── tpch6.result │ ├── tpch7.result │ ├── tpch8.result │ └── tpch9.result │ ├── 0.01G_z1 │ ├── theta_hyracks.result │ ├── theta_lines_self_join.result │ ├── theta_lines_self_join_input_dominated.result │ ├── theta_tpch5_R_N_S_L.result │ └── theta_tpch7_L_S_N1.result │ ├── 0.01G_z2 │ ├── theta_hyracks.result │ ├── theta_lines_self_join.result │ ├── theta_lines_self_join_input_dominated.result │ ├── theta_tpch5_R_N_S_L.result │ └── theta_tpch7_L_S_N1.result │ ├── 0.01G_z3 │ ├── theta_hyracks.result │ ├── theta_lines_self_join.result │ ├── theta_lines_self_join_input_dominated.result │ ├── theta_tpch5_R_N_S_L.result │ └── theta_tpch7_L_S_N1.result │ ├── 0.01G_z4 │ ├── theta_hyracks.result │ ├── theta_lines_self_join.result │ ├── theta_lines_self_join_input_dominated.result │ ├── theta_tpch5_R_N_S_L.result │ └── theta_tpch7_L_S_N1.result │ └── 0.1G │ ├── distinct_hyracks.result │ ├── hyracks.result │ ├── hyracks_l3_batch.result │ ├── hyracks_pre_agg.result │ ├── theta_hyracks.result │ ├── theta_input_dominated.result │ ├── theta_multiple_join.result │ ├── theta_output_dominated.result │ ├── theta_tpch7.result │ ├── tpch10.result │ ├── tpch19.result │ ├── tpch3.result │ ├── tpch4.result │ ├── tpch5.result │ ├── tpch7.result │ ├── tpch8.result │ └── tpch9.result ├── squall ├── confs │ ├── cluster │ │ ├── 40G_hyracks │ │ ├── 40G_tpch3 │ │ ├── 40G_tpch7 │ │ └── 40G_z0_hyracks_16J_1_11 │ ├── local │ │ ├── 0_01G_distinct_hyracks_ncl │ │ ├── 0_01G_hyracks_irb │ │ ├── 0_01G_hyracks_is │ │ ├── 0_01G_hyracks_ncl │ │ ├── 0_01G_hyracks_nmcl │ │ ├── 0_01G_hyracks_nmpl │ │ ├── 0_01G_hyracks_nrl │ │ ├── 0_01G_tpch10_irb │ │ ├── 0_01G_tpch10_ncl │ │ ├── 0_01G_tpch10_nmcl │ │ ├── 0_01G_tpch10_nrl │ │ ├── 0_01G_tpch12_ncl │ │ ├── 0_01G_tpch3_irb │ │ ├── 0_01G_tpch3_is │ │ ├── 0_01G_tpch3_ncl │ │ ├── 0_01G_tpch3_nmcl │ │ ├── 0_01G_tpch3_nrl │ │ ├── 0_01G_tpch4_ncl │ │ ├── 0_01G_tpch5_irb │ │ ├── 0_01G_tpch5_ncl │ │ ├── 0_01G_tpch5_nmcl │ │ ├── 0_01G_tpch5_nrl │ │ ├── 0_01G_tpch6_ncl │ │ ├── 0_01G_tpch7_irb │ │ ├── 0_01G_tpch7_ncl │ │ ├── 0_01G_tpch7_nmcl │ │ ├── 0_01G_tpch7_nrl │ │ ├── 0_01G_tpch8_irb │ │ ├── 0_01G_tpch8_ncl │ │ ├── 0_01G_tpch8_nmcl │ │ ├── 0_01G_tpch8_nrl │ │ ├── 0_01G_tpch9_irb │ │ ├── 0_01G_tpch9_ncl │ │ ├── 0_01G_tpch9_nmcl │ │ └── 0_01G_tpch9_nrl │ └── manual_batching │ │ ├── cluster │ │ ├── B16_10G_hyracks │ │ ├── B16_10G_tpch3 │ │ ├── B16_10G_tpch5 │ │ ├── B1K_10G_hyracks │ │ ├── B1K_10G_tpch10 │ │ ├── B1K_10G_tpch12 │ │ ├── B1K_10G_tpch3 │ │ ├── B1K_10G_tpch4 │ │ ├── B1K_10G_tpch5 │ │ ├── B1K_10G_tpch6 │ │ ├── B1K_10G_tpch9 │ │ ├── B1_10G_hyracks │ │ ├── B1_10G_tpch3 │ │ ├── B1_10G_tpch5 │ │ ├── B256_10G_hyracks │ │ ├── B256_10G_tpch3 │ │ ├── B256_10G_tpch5 │ │ ├── B4K_10G_hyracks │ │ ├── B4_10G_hyracks │ │ ├── B4_10G_tpch3 │ │ ├── B4_10G_tpch5 │ │ ├── B4k_10G_tpch3 │ │ ├── B4k_10G_tpch5 │ │ ├── B64_10G_hyracks │ │ ├── B64_10G_tpch3 │ │ └── B64_10G_tpch5 │ │ └── local │ │ ├── 0_01G_hyracks_local │ │ ├── 0_01G_tpch10_local │ │ ├── 0_01G_tpch12_local │ │ ├── 0_01G_tpch3_local │ │ ├── 0_01G_tpch4_local │ │ ├── 0_01G_tpch5_local │ │ ├── 0_01G_tpch6_local │ │ └── 0_01G_tpch9_local ├── schemas │ ├── Ex1.txt │ ├── Ex2.txt │ ├── rst.txt │ └── tpch.txt └── sql_queries │ ├── distinct_hyracks.sql │ ├── hyracks.sql │ ├── misc.sql │ ├── tpch10.sql │ ├── tpch12.sql │ ├── tpch19.sql │ ├── tpch3.sql │ ├── tpch4.sql │ ├── tpch5.sql │ ├── tpch6.sql │ ├── tpch7.sql │ ├── tpch8.sql │ └── tpch9.sql └── squall_plan_runner └── confs ├── cluster ├── 10G_dbtoaster_hash_hypercube_hyracks ├── 10G_dbtoaster_hash_hypercube_hyracks_redis ├── 10G_dbtoaster_hash_hypercube_tpch10 ├── 10G_dbtoaster_hash_hypercube_tpch3 ├── 10G_dbtoaster_hash_hypercube_tpch5 ├── 10G_dbtoaster_hash_hypercube_tpch9 ├── 10G_dbtoaster_hash_hypercube_tpch9_partial ├── 10G_dbtoaster_hybrid_hypercube_hyracks ├── 10G_dbtoaster_hybrid_hypercube_manual_hyracks ├── 10G_dbtoaster_hybrid_hypercube_tpch3 ├── 10G_dbtoaster_hybrid_hypercube_tpch5 ├── 10G_dbtoaster_hybrid_hypercube_tpch9 ├── 10G_dbtoaster_hybrid_hypercube_tpch9_partial ├── 10G_dbtoaster_hyracks ├── 10G_dbtoaster_seqjoin_tpch3 ├── 10G_dbtoaster_seqjoin_tpch3_hash ├── 10G_dbtoaster_seqjoin_tpch5 ├── 10G_dbtoaster_seqjoin_tpch5_hash ├── 10G_dbtoaster_tpch10 ├── 10G_dbtoaster_tpch3 ├── 10G_dbtoaster_tpch5 ├── 10G_dbtoaster_tpch9 ├── 10G_dbtoaster_tpch9_partial ├── 10G_dbtoaster_z1_hyracks ├── 10G_theta_hyracks ├── 10G_traditional_hash_hypercube_hyracks ├── 10G_traditional_hash_hypercube_tpch3 ├── 10G_traditional_hash_hypercube_tpch9_partial ├── 10G_traditional_hybrid_hypercube_hyracks ├── 10G_traditional_hybrid_hypercube_tpch3 ├── 10G_traditional_hybrid_hypercube_tpch9_partial ├── 10G_traditional_random_hypercube_hyracks ├── 10G_traditional_random_hypercube_tpch3 ├── 10G_traditional_random_hypercube_tpch9_partial ├── 10G_traditional_tpch5 ├── 10G_z0_hyracks ├── 10G_z0_tpch3_4 ├── 10G_z0_tpch7_4 ├── 10G_z1_hyracks ├── 1G_dbtoaster_hash_hypercube_tpch5 ├── 1G_hyracks ├── dbtoaster_google_failed ├── dbtoaster_google_failed_traditional ├── dbtoaster_hash_hypercube_reachability ├── dbtoaster_hash_hypercube_reachability_seq ├── dbtoaster_hash_hypercube_url_reachability ├── dbtoaster_hash_hypercube_url_reachability_skewed ├── dbtoaster_hybrid_hypercube_url_reachability ├── dbtoaster_hybrid_hypercube_url_reachability_skewed ├── dbtoaster_reachability └── dbtoaster_reachability_seq ├── extra-local ├── 0_01G_test_sync ├── 0_01G_test_sync2 ├── 0_01G_theta_output_dominated ├── 0_01G_tpch7HDFS_Materializer └── 0_01G_tpch7HDFS_Source ├── local ├── 0_01G_hyracks ├── 0_01G_hyracks_l3_batch ├── 0_01G_hyracks_pre_agg ├── 0_01G_scalahyracks ├── 0_01G_theta_hyracks ├── 0_01G_theta_input_dominated ├── 0_01G_theta_multiple_join ├── 0_01G_theta_tpch10 ├── 0_01G_theta_tpch3 ├── 0_01G_theta_tpch4 ├── 0_01G_theta_tpch5 ├── 0_01G_theta_tpch7 ├── 0_01G_theta_tpch8 ├── 0_01G_theta_tpch9 ├── 0_01G_tpch10 ├── 0_01G_tpch3 ├── 0_01G_tpch4 ├── 0_01G_tpch5 ├── 0_01G_tpch5avg ├── 0_01G_tpch7 ├── 0_01G_tpch8 ├── 0_01G_tpch9 ├── 0_01G_traditional_hash_hypercube_hyracks ├── 0_01G_traditional_hash_hypercube_tpch3 ├── 0_01G_traditional_hybrid_hypercube_hyracks ├── 0_01G_traditional_hyracks ├── 0_01G_traditional_random_hypercube_hyracks ├── 0_01G_traditional_tpch3 ├── 0_01G_traditional_tpch5 ├── 10K_rst ├── traditional_reachability └── traditional_url_reachability ├── low_selectivity ├── ewh_sample │ └── local │ │ └── eclipse-run │ │ ├── 0_01G_lineitem_self_input │ │ ├── 0_01G_theta_line_self_join │ │ ├── 0_01G_theta_tpch7_L_S_N1 │ │ └── sample_peer_self ├── push │ └── local │ │ ├── console-joiners │ │ ├── 0_01G_theta_hyracks_16J │ │ ├── 0_01G_theta_hyracks_32J │ │ ├── 0_01G_theta_hyracks_64J │ │ ├── 0_01G_theta_hyracks_8J │ │ ├── 0_01G_theta_hyracks_l128J │ │ ├── 0_01G_z1_theta_line_self_join_16J │ │ ├── 0_01G_z1_theta_line_self_join_32J │ │ ├── 0_01G_z1_theta_line_self_join_64J │ │ ├── 0_01G_z1_theta_line_self_join_8J │ │ └── 0_01G_z1_theta_line_self_join_l128J │ │ ├── console-run │ │ ├── 0_01G_theta_hyracks │ │ ├── 0_01G_theta_line_self_join │ │ ├── 0_01G_theta_line_self_join_input_dominated │ │ ├── 0_01G_theta_tpch5_R_N_S_L │ │ ├── 0_01G_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z1_theta_hyracks │ │ ├── 0_01G_z1_theta_line_self_join │ │ ├── 0_01G_z1_theta_line_self_join_input_dominated │ │ ├── 0_01G_z1_theta_tpch5_R_N_S_L │ │ ├── 0_01G_z1_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z2_theta_hyracks │ │ ├── 0_01G_z2_theta_line_self_join │ │ ├── 0_01G_z2_theta_line_self_join_input_dominated │ │ ├── 0_01G_z2_theta_tpch5_R_N_S_L │ │ ├── 0_01G_z2_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z3_theta_hyracks │ │ ├── 0_01G_z3_theta_line_self_join │ │ ├── 0_01G_z3_theta_line_self_join_input_dominated │ │ ├── 0_01G_z3_theta_tpch5_R_N_S_L │ │ ├── 0_01G_z3_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z4_theta_hyracks │ │ ├── 0_01G_z4_theta_line_self_join │ │ ├── 0_01G_z4_theta_line_self_join_input_dominated │ │ ├── 0_01G_z4_theta_tpch5_R_N_S_L │ │ └── 0_01G_z4_theta_tpch7_L_S_N1 │ │ ├── console │ │ ├── 0_01G_theta_hyracks │ │ ├── 0_01G_theta_line_self_join │ │ ├── 0_01G_theta_line_self_join_input_dominated │ │ ├── 0_01G_theta_tpch5_R_N_S_L │ │ ├── 0_01G_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z1_theta_hyracks │ │ ├── 0_01G_z1_theta_line_self_join │ │ ├── 0_01G_z1_theta_line_self_join_input_dominated │ │ ├── 0_01G_z1_theta_tpch5_R_N_S_L │ │ ├── 0_01G_z1_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z2_theta_hyracks │ │ ├── 0_01G_z2_theta_line_self_join │ │ ├── 0_01G_z2_theta_line_self_join_input_dominated │ │ ├── 0_01G_z2_theta_tpch5_R_N_S_L │ │ ├── 0_01G_z2_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z3_theta_hyracks │ │ ├── 0_01G_z3_theta_line_self_join │ │ ├── 0_01G_z3_theta_line_self_join_input_dominated │ │ ├── 0_01G_z3_theta_tpch5_R_N_S_L │ │ ├── 0_01G_z3_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z4_theta_hyracks │ │ ├── 0_01G_z4_theta_line_self_join │ │ ├── 0_01G_z4_theta_line_self_join_input_dominated │ │ ├── 0_01G_z4_theta_tpch5_R_N_S_L │ │ └── 0_01G_z4_theta_tpch7_L_S_N1 │ │ └── eclipse-run │ │ ├── 0_01G_theta_hyracks │ │ ├── 0_01G_theta_line_self_join │ │ ├── 0_01G_theta_line_self_join_input_dominated │ │ ├── 0_01G_theta_tpch5_R_N_S_L │ │ ├── 0_01G_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z1_theta_hyracks │ │ ├── 0_01G_z1_theta_line_self_join │ │ ├── 0_01G_z1_theta_line_self_join_input_dominated │ │ ├── 0_01G_z1_theta_tpch5_R_N_S_L │ │ ├── 0_01G_z1_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z2_theta_hyracks │ │ ├── 0_01G_z2_theta_line_self_join │ │ ├── 0_01G_z2_theta_line_self_join_input_dominated │ │ ├── 0_01G_z2_theta_tpch5_R_N_S_L │ │ ├── 0_01G_z2_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z3_theta_hyracks │ │ ├── 0_01G_z3_theta_line_self_join │ │ ├── 0_01G_z3_theta_line_self_join_input_dominated │ │ ├── 0_01G_z3_theta_tpch5_R_N_S_L │ │ ├── 0_01G_z3_theta_tpch7_L_S_N1 │ │ ├── 0_01G_z4_theta_hyracks │ │ ├── 0_01G_z4_theta_line_self_join │ │ ├── 0_01G_z4_theta_line_self_join_input_dominated │ │ ├── 0_01G_z4_theta_tpch5_R_N_S_L │ │ └── 0_01G_z4_theta_tpch7_L_S_N1 └── sample │ └── local │ ├── eclipse-generic │ ├── 0_01G_theta_line_self_join │ └── 0_01G_theta_tpch7_L_S_N1 │ └── eclipse-run │ ├── 0_01G_theta_line_self_join │ └── 0_01G_theta_tpch7_L_S_N1 ├── manual_batching ├── config_template.txt ├── latency │ └── local │ │ ├── 0_01G_theta_tpch5_test │ │ ├── 10G_uniform_static_naive_bnci │ │ └── 10G_uniform_static_opt_bnci ├── parallel │ ├── formula │ │ ├── 10G_hyracks_parallel │ │ ├── 10K_rst_parallel │ │ ├── 1G_hyracksPreAgg_parallel │ │ ├── 1G_hyracks_parallel │ │ ├── 1G_tpch3_parallel_4 │ │ ├── 1G_tpch7_parallel_4 │ │ ├── 5G_hyracks_parallel │ │ ├── 5G_hyracks_parallel_t2 │ │ └── 8G_tpch7_parallel_1 │ └── guess │ │ ├── 1G_tpch7_parallel_16_formula │ │ ├── 1G_tpch7_parallel_16_opt │ │ ├── 1G_tpch7_parallel_16_opt_32ACK │ │ ├── 1G_tpch7_parallel_1_20 │ │ ├── 1G_tpch7_parallel_1_20_F1 │ │ ├── 1G_tpch7_parallel_1_20_F16 │ │ ├── 1G_tpch7_parallel_1_20_F3 │ │ ├── 1G_tpch7_parallel_1_2x │ │ ├── 1G_tpch7_parallel_2_20 │ │ ├── 1G_tpch7_parallel_2_20_8ACK │ │ ├── 1G_tpch7_parallel_2_20_BB │ │ ├── 1G_tpch7_parallel_2_20_SB │ │ ├── 1G_tpch7_parallel_2_20_formula │ │ ├── 1G_tpch7_parallel_2_2x │ │ ├── 1G_tpch7_parallel_4_20 │ │ ├── 1G_tpch7_parallel_4_20_formula │ │ ├── 1G_tpch7_parallel_4_2x │ │ ├── 1G_tpch7_parallel_4_2xx │ │ ├── 1G_tpch7_parallel_8_20 │ │ ├── 1G_tpch7_parallel_8_2EQ │ │ ├── 1G_tpch7_parallel_8_2x │ │ ├── 1G_tpch7_parallel_8_2xx │ │ ├── 1G_tpch7_parallel_8_2xxx │ │ ├── 1G_tpch7_parallel_8_opt │ │ ├── 1G_tpch7_parallel_old │ │ ├── 4G_hyracks_parallel_t1 │ │ ├── 5G_hyracksPreAgg_parallel_t1 │ │ └── 5G_hyracksPreAgg_parallel_t2 └── serial │ ├── 0.01G_hyracks_pre_agg_serial │ ├── 0.01G_hyracks_serial │ ├── 0.01G_theta_hyracks_serial │ ├── 0.01G_theta_input_dominated_serial │ ├── 0.01G_theta_multiple_join_serial │ ├── 0.01G_theta_output_dominated_serial │ ├── 0.01G_theta_tpch7_serial │ ├── 0.01G_tpch10_serial │ ├── 0.01G_tpch3_serial │ ├── 0.01G_tpch4_serial │ ├── 0.01G_tpch5_serial │ ├── 0.01G_tpch7_serial │ ├── 0.01G_tpch8_serial │ ├── 0.01G_tpch9_serial │ └── 10K_rst_serial └── squall-ui ├── google_failed_h1_hash-local1_dbtoaster-8 ├── google_failed_h1_hash-local1_traditional-8 ├── google_failed_h1_hybrid-local1_dbtoaster-8 ├── google_failed_h1_hybrid-local1_traditional-8 ├── google_failed_h1_random-local1_dbtoaster-8 ├── google_failed_h1_random-local1_traditional-8 ├── hyrakcs_h1_hash-local1_dbtoaster-16 ├── hyrakcs_h1_hash-local1_traditional-16 ├── hyrakcs_h1_hybrid-local1_dbtoaster-16 ├── hyrakcs_h1_hybrid-local1_traditional-16 ├── hyrakcs_h1_random-local1_dbtoaster-16 ├── hyrakcs_h1_random-local1_traditional-16 ├── reachability_h1_hash-local1_dbtoaster-36 ├── reachability_h1_hash-local1_traditional-36 ├── reachability_h1_hybrid-local1_dbtoaster-36 ├── reachability_h1_hybrid-local1_traditional-36 ├── reachability_h1_random-local1_dbtoaster-36 ├── reachability_h1_random-local1_traditional-36 ├── seq_reachability_h1_hash-local1_dbtoaster-h2_hash-local2_dbtoaster-36 ├── seq_reachability_h1_hash-local1_dbtoaster-h2_hybrid-local2_dbtoaster-36 ├── seq_reachability_h1_hash-local1_dbtoaster-h2_random-local2_dbtoaster-36 ├── seq_reachability_h1_hybrid-local1_dbtoaster-h2_hash-local2_dbtoaster-36 ├── seq_reachability_h1_hybrid-local1_dbtoaster-h2_hybrid-local2_dbtoaster-36 ├── seq_reachability_h1_random-local1_dbtoaster-h2_hash-local2_dbtoaster-36 ├── seq_reachability_h1_random-local1_dbtoaster-h2_hybrid-local2_dbtoaster-36 ├── seq_reachability_h1_random-local1_dbtoaster-h2_random-local2_dbtoaster-36 ├── tpch9_partial_h1_hash-local1_dbtoaster-8 ├── tpch9_partial_h1_hash-local1_traditional-8 ├── tpch9_partial_h1_hybrid-local1_dbtoaster-8 ├── tpch9_partial_h1_hybrid-local1_traditional-8 ├── tpch9_partial_h1_random-local1_dbtoaster-8 ├── tpch9_partial_h1_random-local1_traditional-8 ├── url_reachability_skewed_h1_hash-local1_dbtoaster-40 ├── url_reachability_skewed_h1_hash-local1_traditional-40 ├── url_reachability_skewed_h1_hybrid-local1_dbtoaster-40 ├── url_reachability_skewed_h1_hybrid-local1_traditional-40 ├── url_reachability_skewed_h1_random-local1_dbtoaster-40 └── url_reachability_skewed_h1_random-local1_tradtional-40 /.gitignore: -------------------------------------------------------------------------------- 1 | # Extensions 2 | *.class 3 | *.swp 4 | 5 | # Paths 6 | .idea 7 | cbin 8 | dbtoaster 9 | eclipse 10 | resources 11 | experiments 12 | storm-0* 13 | apache-storm-* 14 | test/squall_plan_runner/confs/special_purpose 15 | contrib/0.2.5 16 | 17 | test/m_bucket 18 | test/data/tpch/small 19 | test/data/tpch/0.1G 20 | test/data/tpch/0.01G_z1 21 | test/data/tpch/0.01G_z2 22 | test/data/tpch/0.01G_z3 23 | test/data/tpch/0.01G_z4 24 | test/data/tpch/0.05G 25 | test/data/tpch/1G 26 | test/data/tpch/4G 27 | test/data/tpch/10G 28 | test/data/tpch/50G 29 | test/results/tpch/10G 30 | test/results/tpch/50G 31 | 32 | # sbt 33 | target/ 34 | cbin 35 | dbtoaster 36 | 37 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | 3 | scala: 4 | - 2.11.6 5 | 6 | before_script: 7 | - sudo chmod +x /usr/local/bin/sbt 8 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | EPFLDATA/Squall 2 | Copyright 2011-2015 The Squall Collaboration: 3 | -- the EPFL DATA Laboratory (http://data.epfl.ch) and 4 | -- students of the 2014 EPFL Big Data course: 5 | -- Daniel Espino (@akathorn) 6 | -- Michalis Zervos (@mzervos) 7 | -- Matthaios-Alexandros Olma (@amolma) 8 | -- Andriani Stylianou (@antrianis) 9 | 10 | This product includes software developed by ... 11 | 12 | (See the contrib folder) 13 | 14 | -------------------------------------------------------------------------------- /bin/install.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | . ./storm_env.sh 4 | 5 | # The following is needed only if we use DBToaster operators 6 | # On Linux machines, we need to install Scala (at least 2.10) 7 | # wget http://www.scala-lang.org/files/archive/scala-2.10.4.deb 8 | # sudo dpkg -i scala-2.10.4.deb 9 | # sudo apt-get update 10 | # sudo apt-get install scala 11 | 12 | # Downloading Storm and putting it to the right place 13 | echo "Downloading and extracting $STORMNAME ..." 14 | wget http://mirror.easyname.ch/apache/storm/$STORMNAME/$STORMNAME.tar.gz 15 | tar -xzf $STORMNAME.tar.gz 16 | mv $STORMNAME .. 17 | rm $STORMNAME.tar.gz 18 | 19 | # Compiling Squall and generating dependencies 20 | echo "Compiling Squall and generating dependencies ..." 21 | CURR_DIR=`pwd` 22 | cd .. 23 | sbt package 24 | sbt assemblyPackageDependency 25 | cd $CURR_DIR 26 | # The following is used only for the Cluster Mode 27 | cp ../squall-core/target/squall-dependencies-0.2.0.jar ../$STORMNAME/lib/ 28 | -------------------------------------------------------------------------------- /bin/recompile.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | cd ../ 4 | 5 | #sbt clean 6 | 7 | # Generate squall-0.2.0.jar 8 | sbt package 9 | 10 | # Generate squall-dependencies-0.2.0.jar 11 | #sbt assemblyPackageDependency 12 | 13 | # Generate squall-standalone-0.2.0.jar 14 | #sbt assembly 15 | -------------------------------------------------------------------------------- /bin/squall_cluster.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | . ./storm_env.sh 3 | 4 | printFormat (){ 5 | echo "Format: ./squall_cluster.sh MODE CONFIG_PATH" 6 | echo " MODE: SQL (default) or PLAN_RUNNER" 7 | } 8 | 9 | # Check the number of arguments 10 | if [[ $# -lt 1 || $# -gt 2 ]]; then 11 | echo "ERROR:: Improper number of arguments!" 12 | printFormat 13 | exit 1 14 | fi 15 | 16 | # Reading input paramters 17 | MODE=$1 18 | if [[ "$MODE" != "PLAN_RUNNER" && "$MODE" != "SQL" ]]; then 19 | MODE=SQL 20 | else 21 | shift 22 | fi 23 | 24 | # Main class 25 | if [ "$MODE" == "PLAN_RUNNER" ]; then 26 | MAIN_CLASS=ch.epfl.data.squall.main.Main 27 | else 28 | MAIN_CLASS=ch.epfl.data.squall.api.sql.main.ParserMain 29 | fi 30 | 31 | # Set config file path and check if it exist 32 | if [[ $# -lt 1 ]]; then 33 | echo "ERROR:: Missing configuration file path!" 34 | printFormat 35 | exit 1 36 | fi 37 | CONFIG_PATH=$1 38 | if ! [ -f $CONFIG_PATH ]; then 39 | echo "ERROR:: File $CONFIG_PATH does not exist! Please specify a valid configuration file!" 40 | exit 1 41 | fi 42 | 43 | # Running 44 | ../$STORMNAME/bin/storm jar ../squall-core/target/squall-0.2.0.jar $MAIN_CLASS $CONFIG_PATH 45 | -------------------------------------------------------------------------------- /bin/squall_local.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | . ./storm_env.sh 3 | 4 | printFormat (){ 5 | echo "Format: ./squall_local.sh MODE CONFIG_PATH" 6 | echo " MODE: SQL (default) or PLAN_RUNNER" 7 | } 8 | 9 | # Check the number of arguments 10 | if [[ $# -lt 1 || $# -gt 2 ]]; then 11 | echo "ERROR:: Improper number of arguments!" 12 | printFormat 13 | exit 1 14 | fi 15 | 16 | # Reading input paramters 17 | MODE=$1 18 | if [[ "$MODE" != "PLAN_RUNNER" && "$MODE" != "SQL" ]]; then 19 | MODE=SQL 20 | else 21 | shift 22 | fi 23 | 24 | # Main class 25 | if [ "$MODE" == "PLAN_RUNNER" ]; then 26 | MAIN_CLASS=ch.epfl.data.squall.main.Main 27 | else 28 | MAIN_CLASS=ch.epfl.data.squall.api.sql.main.ParserMain 29 | fi 30 | 31 | # Set config file path and check if it exist 32 | if [[ $# -lt 1 ]]; then 33 | echo "ERROR:: Missing configuration file path!" 34 | printFormat 35 | exit 1 36 | fi 37 | CONFIG_PATH=$1 38 | if ! [ -f $CONFIG_PATH ]; then 39 | echo "ERROR:: File $CONFIG_PATH does not exist! Please specify a valid configuration file!" 40 | exit 1 41 | fi 42 | 43 | # Running 44 | java -Xmx128m -cp ../squall-core/target/squall-0.2.0.jar:../squall-core/target/squall-dependencies-0.2.0.jar:../$STORMNAME/lib/* $MAIN_CLASS $CONFIG_PATH 45 | -------------------------------------------------------------------------------- /bin/storm_env.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | #STORMNAME=storm-0.9.2-incubating 4 | STORMNAME=apache-storm-0.9.4 5 | 6 | # DBTOASTER installation folder. Only required if QueryPlan uses DBToasterJoinComponent 7 | DBTOASTER_HOME=../target/dbtoaster 8 | export DBTOASTER_HOME=$DBTOASTER_HOME 9 | -------------------------------------------------------------------------------- /contrib/bheaven-0.0.3.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/epfldata/squall/9b5c9f14ead726e7e7e6c452598f488657a5be18/contrib/bheaven-0.0.3.jar -------------------------------------------------------------------------------- /contrib/storm-graphite-0.2.4-SNAPSHOT-all.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/epfldata/squall/9b5c9f14ead726e7e7e6c452598f488657a5be18/contrib/storm-graphite-0.2.4-SNAPSHOT-all.jar -------------------------------------------------------------------------------- /contrib/ujmp-complete-0.2.5.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/epfldata/squall/9b5c9f14ead726e7e7e6c452598f488657a5be18/contrib/ujmp-complete-0.2.5.jar -------------------------------------------------------------------------------- /logo/logo.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/epfldata/squall/9b5c9f14ead726e7e7e6c452598f488657a5be18/logo/logo.jpg -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=0.13.8 2 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.13.0") 2 | -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/api/sql/estimators/SelectivityEstimator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.api.sql.estimators; 21 | 22 | import net.sf.jsqlparser.expression.Expression; 23 | 24 | public interface SelectivityEstimator { 25 | 26 | public double estimate(Expression expr); 27 | 28 | } 29 | -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/api/sql/optimizers/Optimizer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.api.sql.optimizers; 21 | 22 | import ch.epfl.data.squall.query_plans.QueryBuilder; 23 | 24 | public interface Optimizer { 25 | 26 | public QueryBuilder generate(); 27 | 28 | } -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/api/sql/util/NotFromMyBranchException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.api.sql.util; 21 | 22 | public class NotFromMyBranchException extends RuntimeException { 23 | 24 | /** 25 | * 26 | */ 27 | private static final long serialVersionUID = 1L; 28 | 29 | } 30 | -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/ewh/data_structures/ListAdapter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.ewh.data_structures; 21 | 22 | public interface ListAdapter> { 23 | public void add(T t); 24 | 25 | public void set(int index, T t); 26 | 27 | public T get(int index); 28 | 29 | public void remove(int index); 30 | 31 | public void sort(); 32 | 33 | public int size(); 34 | } -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/ewh/visualize/VisualizerInterface.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.ewh.visualize; 21 | 22 | import ch.epfl.data.squall.ewh.data_structures.UJMPAdapterByteMatrix; 23 | import ch.epfl.data.squall.ewh.data_structures.UJMPAdapterIntMatrix; 24 | 25 | /* 26 | * Visitor interface 27 | */ 28 | public interface VisualizerInterface { 29 | 30 | public void visualize(UJMPAdapterByteMatrix m); 31 | 32 | public void visualize(UJMPAdapterIntMatrix m); 33 | 34 | } -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/storm_components/SignaledDataSourceComponentInterface.java: -------------------------------------------------------------------------------- 1 | package ch.epfl.data.squall.storm_components; 2 | 3 | //This is just a proxy for SynchronizedStormDataSource which is out of the squall-core 4 | public interface SignaledDataSourceComponentInterface { 5 | 6 | } -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/storm_components/StormEmitter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.storm_components; 21 | 22 | public interface StormEmitter { 23 | public String[] getEmitterIDs(); 24 | 25 | public String getInfoID(); 26 | 27 | public String getName(); 28 | } -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/storm_components/SynchronizedStormDataSourceInterface.java: -------------------------------------------------------------------------------- 1 | package ch.epfl.data.squall.storm_components; 2 | 3 | // This is just a proxy for SynchronizedStormDataSource which is out of the squall-core 4 | public interface SynchronizedStormDataSourceInterface { 5 | 6 | public static String SHUFFLE_GROUPING_STREAMID = "sync_shuffle"; 7 | 8 | } 9 | -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/types/Type.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.types; 21 | 22 | import java.io.Serializable; 23 | 24 | public interface Type extends Serializable { 25 | public T fromString(String str); 26 | 27 | // bigger - smaller 28 | public double getDistance(T bigger, T smaller); 29 | 30 | public T getInitialValue(); 31 | 32 | public String toString(T obj); 33 | 34 | public T generateRandomInstance(); 35 | } -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/utilities/CustomReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.utilities; 21 | 22 | import java.io.IOException; 23 | import java.io.Serializable; 24 | import java.io.InputStream; 25 | 26 | public interface CustomReader extends Serializable { 27 | 28 | public void close(); 29 | 30 | public String readLine() throws IOException; 31 | } 32 | -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/utilities/PartitioningScheme.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * * Copyright (c) 2011-2015 EPFL DATA Laboratory 4 | * * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 5 | * * 6 | * * All rights reserved. 7 | * * 8 | * * Licensed under the Apache License, Version 2.0 (the "License"); 9 | * * you may not use this file except in compliance with the License. 10 | * * You may obtain a copy of the License at 11 | * * 12 | * * http://www.apache.org/licenses/LICENSE-2.0 13 | * * 14 | * * Unless required by applicable law or agreed to in writing, software 15 | * * distributed under the License is distributed on an "AS IS" BASIS, 16 | * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * * See the License for the specific language governing permissions and 18 | * * limitations under the License. 19 | * 20 | */ 21 | 22 | package ch.epfl.data.squall.utilities; 23 | 24 | public enum PartitioningScheme { 25 | BRUTEFORCEHYBRIDHYPERCUBE, 26 | MANUALHYBRIDHYPERCUBE, 27 | HASHHYPERCUBE, 28 | HYPERCUBE, 29 | HASH, 30 | STARSCHEMA 31 | } 32 | -------------------------------------------------------------------------------- /squall-core/src/main/java/ch/epfl/data/squall/utilities/ReaderProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.utilities; 21 | 22 | import java.io.Serializable; 23 | 24 | public abstract class ReaderProvider implements Serializable { 25 | private static final long serialVersionUID = 1L; 26 | 27 | public abstract boolean canProvide (SquallContext context, String name); 28 | 29 | public abstract CustomReader getReaderForName (String name, int fileSection, int fileParts); 30 | 31 | } 32 | -------------------------------------------------------------------------------- /squall-core/src/test/resources/logback-test.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/RSTTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test 21 | 22 | class RSTTest extends TestSuite { 23 | test("10K_rst") { 24 | val query = "10K_rst" 25 | val result = runQuery(query) 26 | assert(result.equals(expectedResultFor(result, query))) 27 | } 28 | 29 | } 30 | 31 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/sql/SqlHyracksTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test.sql 21 | 22 | import ch.epfl.data.squall.test.TestSuite 23 | 24 | class SqlHyracksTest extends TestSuite { 25 | 26 | testSQL("0_01G_distinct_hyracks_ncl") 27 | testSQL("0_01G_hyracks_irb") 28 | testSQL("0_01G_hyracks_is") 29 | testSQL("0_01G_hyracks_ncl") 30 | testSQL("0_01G_hyracks_nmcl") 31 | testSQL("0_01G_hyracks_nmpl") 32 | testSQL("0_01G_hyracks_nrl") 33 | 34 | 35 | } 36 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/sql/SqlTpch10Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test.sql 21 | 22 | import ch.epfl.data.squall.test.TestSuite 23 | 24 | class SqlTpch10Test extends TestSuite { 25 | 26 | testSQL("0_01G_tpch10_irb") 27 | testSQL("0_01G_tpch10_ncl") 28 | testSQL("0_01G_tpch10_nmcl") 29 | testSQL("0_01G_tpch10_nrl") 30 | 31 | } 32 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/sql/SqlTpch12Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test.sql 21 | 22 | import ch.epfl.data.squall.test.TestSuite 23 | 24 | class SqlTpch12Test extends TestSuite { 25 | 26 | testSQL("0_01G_tpch12_ncl") 27 | 28 | } 29 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/sql/SqlTpch3Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test.sql 21 | 22 | import ch.epfl.data.squall.test.TestSuite 23 | 24 | class SqlTpch3Test extends TestSuite { 25 | 26 | testSQL("0_01G_tpch3_irb") 27 | testSQL("0_01G_tpch3_is") 28 | testSQL("0_01G_tpch3_ncl") 29 | testSQL("0_01G_tpch3_nmcl") 30 | testSQL("0_01G_tpch3_nrl") 31 | 32 | } 33 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/sql/SqlTpch4Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test.sql 21 | 22 | import ch.epfl.data.squall.test.TestSuite 23 | 24 | class SqlTpch4Test extends TestSuite { 25 | 26 | testSQL("0_01G_tpch4_ncl") 27 | 28 | } 29 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/sql/SqlTpch5Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test.sql 21 | 22 | import ch.epfl.data.squall.test.TestSuite 23 | 24 | class SqlTpch5Test extends TestSuite { 25 | 26 | testSQL("0_01G_tpch5_irb") 27 | testSQL("0_01G_tpch5_ncl") 28 | testSQL("0_01G_tpch5_nmcl") 29 | testSQL("0_01G_tpch5_nrl") 30 | 31 | 32 | } 33 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/sql/SqlTpch6Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test.sql 21 | 22 | import ch.epfl.data.squall.test.TestSuite 23 | 24 | class SqlTpch6Test extends TestSuite { 25 | 26 | testSQL("0_01G_tpch6_ncl") 27 | 28 | } 29 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/sql/SqlTpch7Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test.sql 21 | 22 | import ch.epfl.data.squall.test.TestSuite 23 | 24 | class SqlTpch7Test extends TestSuite { 25 | 26 | testSQL("0_01G_tpch7_irb") 27 | testSQL("0_01G_tpch7_ncl") 28 | testSQL("0_01G_tpch7_nmcl") 29 | testSQL("0_01G_tpch7_nrl") 30 | 31 | } 32 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/sql/SqlTpch8Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test.sql 21 | 22 | import ch.epfl.data.squall.test.TestSuite 23 | 24 | class SqlTpch8Test extends TestSuite { 25 | 26 | testSQL("0_01G_tpch8_irb") 27 | testSQL("0_01G_tpch8_ncl") 28 | testSQL("0_01G_tpch8_nmcl") 29 | testSQL("0_01G_tpch8_nrl") 30 | 31 | } 32 | -------------------------------------------------------------------------------- /squall-core/src/test/scala/sql/SqlTpch9Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2011-2015 EPFL DATA Laboratory 3 | * Copyright (c) 2014-2015 The Squall Collaboration (see NOTICE) 4 | * 5 | * All rights reserved. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package ch.epfl.data.squall.test.sql 21 | 22 | import ch.epfl.data.squall.test.TestSuite 23 | 24 | class SqlTpch9Test extends TestSuite { 25 | 26 | testSQL("0_01G_tpch9_irb") 27 | testSQL("0_01G_tpch9_ncl") 28 | testSQL("0_01G_tpch9_nmcl") 29 | testSQL("0_01G_tpch9_nrl") 30 | 31 | } 32 | -------------------------------------------------------------------------------- /squall-signals/src/main/java/ch/epfl/data/squall/components/signal_components/SignalUtilities.java: -------------------------------------------------------------------------------- 1 | package ch.epfl.data.squall.components.signal_components; 2 | 3 | public class SignalUtilities { 4 | 5 | public static int DISTRIBUTION_SIGNAL=0; 6 | public static int HARMONIZER_SIGNAL=1; 7 | 8 | 9 | public static byte[] createSignal(int signalNum, byte[] payload){ 10 | 11 | byte[] signal= toBytes(signalNum); 12 | byte[] newArray = new byte[signal.length + payload.length]; 13 | System.arraycopy(signal, 0, newArray, 0, signal.length); 14 | System.arraycopy(payload, 0, newArray, 4, payload.length); 15 | return newArray; 16 | } 17 | 18 | 19 | public static int byteArrayToInt(byte[] b) { 20 | int value = 0; 21 | for (int i = 0; i < 4; i++) { 22 | int shift = (4 - 1 - i) * 8; 23 | value += (b[i] & 0x000000FF) << shift; 24 | } 25 | return value; 26 | } 27 | 28 | public static byte[] toBytes(int i) { 29 | byte[] result = new byte[4]; 30 | result[0] = (byte) (i >> 24); 31 | result[1] = (byte) (i >> 16); 32 | result[2] = (byte) (i >> 8); 33 | result[3] = (byte) (i /* >> 0 */); 34 | return result; 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /squall-signals/src/main/java/ch/epfl/data/squall/components/signal_components/storm/SignalListener.java: -------------------------------------------------------------------------------- 1 | package ch.epfl.data.squall.components.signal_components.storm; 2 | 3 | public interface SignalListener { 4 | void onSignal(byte[] data); 5 | } 6 | -------------------------------------------------------------------------------- /squall-signals/src/main/java/ch/epfl/data/squall/components/signal_components/storm/StandaloneSignalConnection.java: -------------------------------------------------------------------------------- 1 | package ch.epfl.data.squall.components.signal_components.storm; 2 | 3 | import org.apache.storm.curator.framework.CuratorFrameworkFactory; 4 | import org.apache.storm.curator.retry.RetryNTimes; 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | 8 | public class StandaloneSignalConnection extends AbstractSignalConnection { 9 | private static final Logger LOG = LoggerFactory 10 | .getLogger(StandaloneSignalConnection.class); 11 | 12 | private String connectString; 13 | private int zkRetries = 5; 14 | private int zkRetryInterval = 1000; 15 | 16 | public StandaloneSignalConnection(String name, SignalListener listener, 17 | String connectString) { 18 | this.name = name; 19 | this.listener = listener; 20 | this.connectString = connectString; 21 | } 22 | 23 | public void init() throws Exception { 24 | 25 | this.client = CuratorFrameworkFactory 26 | .builder() 27 | .namespace(namespace) 28 | .connectString(connectString) 29 | .retryPolicy( 30 | new RetryNTimes(this.zkRetries, this.zkRetryInterval)) 31 | .build(); 32 | this.client.start(); 33 | super.initWatcher(); 34 | } 35 | 36 | } 37 | -------------------------------------------------------------------------------- /test/data/google/README: -------------------------------------------------------------------------------- 1 | This directory contains a Google cluster trace generated at 2014-11-18 09:58Z. 2 | 3 | For more information see https://code.google.com/p/googleclusterdata/ . 4 | 5 | This directory should contain MD5SUM, SHA1SUM, and SHA256SUM files, which can be 6 | used to verify the contents of this directory. 7 | 8 | The other files are gzip compressed CSV files whose fields are described in 9 | 'schema.csv', which should be in this directory. 10 | -------------------------------------------------------------------------------- /test/data/pavlo_torrent/sample/peersnapshot-01.tbl: -------------------------------------------------------------------------------- 1 | 1|9222|4|0.0000|0.0000|0.0000|0.0000|0|0|0|0|0E-7|2008-10-28 01:57:35| 2 | 2|9223|7|91.0000|247.0000|0.0000|132.7000|0|1331|0|0|0.1361200|2008-10-28 01:57:51| 3 | 3|9224|7|53.8000|582.6000|0.0000|539.6000|0|8054|0|0|0.1350482|2008-10-28 01:57:51| 4 | 4|9221|7|41.1000|59.8000|0.0000|0.0000|0|0|0|0|0E-7|2008-10-28 01:57:51| 5 | 5|9225|7|57.2000|340.6000|0.0000|295.0000|0|3059|0|0|0.1350482|2008-10-28 01:57:52| 6 | 6|9226|7|18.6000|20.8000|0.0000|0.0000|0|0|0|0|0.1382637|2008-10-28 01:57:52| 7 | 7|9224|9|39.0000|323.6000|0.0000|284.6000|0|5907|0|0|0.2908879|2008-10-28 01:58:03| 8 | 8|9223|9|39.0000|313.4000|0.0000|274.4000|0|4680|0|0|0.2908879|2008-10-28 01:58:03| 9 | 9|9227|9|0.0000|0.0000|0.0000|0.0000|0|0|0|0|0.0011682|2008-10-28 01:58:04| 10 | 10|9222|9|39.0000|323.6000|0.0000|284.6000|0|9579|0|0|0.2897196|2008-10-28 01:58:04| 11 | 11|9228|9|39.0000|313.4000|0.0000|274.4000|0|7147|0|0|0.2908879|2008-10-28 01:58:04| 12 | 12|9229|9|0.0000|0.0000|0.0000|0.0000|0|0|0|0|0.2920561|2008-10-28 01:58:04| 13 | -------------------------------------------------------------------------------- /test/data/pavlo_torrent/schema.txt: -------------------------------------------------------------------------------- 1 | "id","peer_id","torrent_snapshot_id","upload_speed","download_speed","payload_upload_speed","payload_download_speed","total_upload","total_download","fail_count","hashfail_count","progress","created" 2 | "1","9222","4","0.0000","0.0000","0.0000","0.0000","0","0","0","0","0E-7","2008-10-28 01:57:35" 3 | -------------------------------------------------------------------------------- /test/data/tpch/0.01G/region.tbl: -------------------------------------------------------------------------------- 1 | 0|AFRICA|lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to | 2 | 1|AMERICA|hs use ironic, even requests. s| 3 | 2|ASIA|ges. thinly even pinto beans ca| 4 | 3|EUROPE|ly final courts cajole furiously final excuse| 5 | 4|MIDDLE EAST|uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl| 6 | -------------------------------------------------------------------------------- /test/results/link_graph/0.01G/reachability_3.result: -------------------------------------------------------------------------------- 1 | 59 = 49 2 | 15 = 8 3 | 37 = 23 4 | 20 = 3 5 | 40 = 49 6 | 64 = 49 7 | 105 = 49 8 | 65 = 49 9 | 81 = 49 10 | 102 = 26 11 | 98 = 93 12 | 48 = 49 13 | 69 = 49 14 | 44 = 1 15 | 22 = 49 16 | 90 = 97 17 | 24 = 49 18 | 25 = 165 19 | 10 = 67 20 | 7 = 61 21 | 31 = 49 22 | 9 = 76 23 | 88 = 3 24 | 76 = 49 25 | -------------------------------------------------------------------------------- /test/results/rst/10K/rst.result: -------------------------------------------------------------------------------- 1 | 2.74286351E8 -------------------------------------------------------------------------------- /test/results/tpch/0.01G/hyracks.result: -------------------------------------------------------------------------------- 1 | BUILDING = 3706 2 | FURNITURE = 3007 3 | MACHINERY = 2536 4 | HOUSEHOLD = 2772 5 | AUTOMOBILE = 2979 6 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/theta_input_dominated.result: -------------------------------------------------------------------------------- 1 | 1.3365971030209991E8 -------------------------------------------------------------------------------- /test/results/tpch/0.01G/theta_lines_self_join.result: -------------------------------------------------------------------------------- 1 | 85671 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/theta_lines_self_join_input_dominated.result: -------------------------------------------------------------------------------- 1 | 2200 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/theta_lines_self_join_mat.result: -------------------------------------------------------------------------------- 1 | 85671 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/theta_multiple_join.result: -------------------------------------------------------------------------------- 1 | 236284.8955 -------------------------------------------------------------------------------- /test/results/tpch/0.01G/theta_output_dominated.result: -------------------------------------------------------------------------------- 1 | 126250.0 -------------------------------------------------------------------------------- /test/results/tpch/0.01G/theta_tpch5_R_N_S_L.result: -------------------------------------------------------------------------------- 1 | 11995 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/theta_tpch7_L_S_N1.result: -------------------------------------------------------------------------------- 1 | 1217 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/tpch12.result: -------------------------------------------------------------------------------- 1 | MAIL|1-URGENT = 50 2 | MAIL|2-HIGH = 64 3 | MAIL|3-MEDIUM = 49 4 | MAIL|4-NOT SPECIFIED = 54 5 | MAIL|5-LOW = 54 6 | SHIP|1-URGENT = 50 7 | SHIP|2-HIGH = 51 8 | SHIP|3-MEDIUM = 52 9 | SHIP|4-NOT SPECIFIED = 52 10 | SHIP|5-LOW = 55 11 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/tpch17.result: -------------------------------------------------------------------------------- 1 | 497.382857 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/tpch19.result: -------------------------------------------------------------------------------- 1 | 2.041626534814003E8 -------------------------------------------------------------------------------- /test/results/tpch/0.01G/tpch4.result: -------------------------------------------------------------------------------- 1 | 2-HIGH = 103 2 | 5-LOW = 128 3 | 3-MEDIUM = 109 4 | 4-NOT SPECIFIED = 102 5 | 1-URGENT = 93 6 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/tpch5.result: -------------------------------------------------------------------------------- 1 | INDONESIA = 566379.5276 2 | VIETNAM = 1000926.6999 3 | CHINA = 740210.757 4 | JAPAN = 660651.2424999999 5 | INDIA = 422874.68439999997 -------------------------------------------------------------------------------- /test/results/tpch/0.01G/tpch5avg.result: -------------------------------------------------------------------------------- 1 | INDONESIA = 566379.5276:16 2 | VIETNAM = 1000926.6999:31 3 | CHINA = 740210.757:21 4 | JAPAN = 660651.2424999999:19 5 | INDIA = 422874.68439999997:16 6 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/tpch6.result: -------------------------------------------------------------------------------- 1 | 1193053.2253 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G/tpch7.result: -------------------------------------------------------------------------------- 1 | GERMANY|FRANCE|1995 = 621159.4882 2 | GERMANY|FRANCE|1996 = 379095.88539999997 3 | FRANCE|GERMANY|1995 = 268068.5774 4 | FRANCE|GERMANY|1996 = 303862.298 -------------------------------------------------------------------------------- /test/results/tpch/0.01G/tpch8.result: -------------------------------------------------------------------------------- 1 | ROMANIA|1996 = 81495.2985 2 | UNITED KINGDOM|1995 = 28554.6996 3 | RUSSIA|1996 = 94173.948 4 | ROMANIA|1995 = 14352.3336 5 | UNITED STATES|1995 = 111623.88200000001 6 | UNITED STATES|1996 = 91758.29199999999 7 | RUSSIA|1995 = 28637.136000000002 8 | EGYPT|1996 = 54531.937999999995 9 | JORDAN|1996 = 32443.933500000003 10 | GERMANY|1995 = 47706.8892 11 | KENYA|1996 = 63040.542400000006 12 | MOZAMBIQUE|1996 = 64998.1277 13 | MOZAMBIQUE|1995 = 17907.1222 14 | PERU|1996 = 58214.380000000005 15 | PERU|1995 = 57581.615000000005 16 | MOROCCO|1995 = 15458.4 17 | JAPAN|1995 = 9492.8636 18 | INDONESIA|1995 = 18722.6684 19 | CHINA|1996 = 51836.3538 20 | ALGERIA|1996 = 27553.5775 21 | SAUDI ARABIA|1995 = 27770.879999999997 -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z1/theta_hyracks.result: -------------------------------------------------------------------------------- 1 | BUILDING = 3063 2 | FURNITURE = 3004 3 | MACHINERY = 2896 4 | HOUSEHOLD = 3048 5 | AUTOMOBILE = 2989 6 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z1/theta_lines_self_join.result: -------------------------------------------------------------------------------- 1 | 157364 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z1/theta_lines_self_join_input_dominated.result: -------------------------------------------------------------------------------- 1 | 1317 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z1/theta_tpch5_R_N_S_L.result: -------------------------------------------------------------------------------- 1 | 22240 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z1/theta_tpch7_L_S_N1.result: -------------------------------------------------------------------------------- 1 | 5619 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z2/theta_hyracks.result: -------------------------------------------------------------------------------- 1 | BUILDING = 3255 2 | FURNITURE = 2525 3 | MACHINERY = 3057 4 | HOUSEHOLD = 3196 5 | AUTOMOBILE = 2967 6 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z2/theta_lines_self_join.result: -------------------------------------------------------------------------------- 1 | 15723 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z2/theta_lines_self_join_input_dominated.result: -------------------------------------------------------------------------------- 1 | 98 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z2/theta_tpch5_R_N_S_L.result: -------------------------------------------------------------------------------- 1 | 32071 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z2/theta_tpch7_L_S_N1.result: -------------------------------------------------------------------------------- 1 | 31300 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z3/theta_hyracks.result: -------------------------------------------------------------------------------- 1 | FURNITURE = 2802 2 | BUILDING = 3769 3 | MACHINERY = 2755 4 | HOUSEHOLD = 2699 5 | AUTOMOBILE = 2975 6 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z3/theta_lines_self_join.result: -------------------------------------------------------------------------------- 1 | 354 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z3/theta_lines_self_join_input_dominated.result: -------------------------------------------------------------------------------- 1 | 5 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z3/theta_tpch5_R_N_S_L.result: -------------------------------------------------------------------------------- 1 | 59537 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z3/theta_tpch7_L_S_N1.result: -------------------------------------------------------------------------------- 1 | 38689 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z4/theta_hyracks.result: -------------------------------------------------------------------------------- 1 | FURNITURE = 2808 2 | BUILDING = 3797 3 | MACHINERY = 2712 4 | HOUSEHOLD = 2715 5 | AUTOMOBILE = 2968 6 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z4/theta_lines_self_join.result: -------------------------------------------------------------------------------- 1 | 3 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z4/theta_lines_self_join_input_dominated.result: -------------------------------------------------------------------------------- 1 | 3 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z4/theta_tpch5_R_N_S_L.result: -------------------------------------------------------------------------------- 1 | 45966 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.01G_z4/theta_tpch7_L_S_N1.result: -------------------------------------------------------------------------------- 1 | 55945 2 | -------------------------------------------------------------------------------- /test/results/tpch/0.1G/distinct_hyracks.result: -------------------------------------------------------------------------------- 1 | BUILDING = 31264 2 | FURNITURE = 29074 3 | MACHINERY = 30341 4 | HOUSEHOLD = 29462 5 | AUTOMOBILE = 29859 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/hyracks.result: -------------------------------------------------------------------------------- 1 | BUILDING = 31264 2 | FURNITURE = 29074 3 | MACHINERY = 30341 4 | HOUSEHOLD = 29462 5 | AUTOMOBILE = 29859 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/hyracks_l3_batch.result: -------------------------------------------------------------------------------- 1 | BUILDING = 31264 2 | FURNITURE = 29074 3 | MACHINERY = 30341 4 | HOUSEHOLD = 29462 5 | AUTOMOBILE = 29859 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/hyracks_pre_agg.result: -------------------------------------------------------------------------------- 1 | BUILDING = 31264.0 2 | FURNITURE = 29074.0 3 | MACHINERY = 30341.0 4 | HOUSEHOLD = 29462.0 5 | AUTOMOBILE = 29859.0 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/theta_hyracks.result: -------------------------------------------------------------------------------- 1 | BUILDING = 31264 2 | FURNITURE = 29074 3 | MACHINERY = 30341 4 | HOUSEHOLD = 29462 5 | AUTOMOBILE = 29859 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/theta_input_dominated.result: -------------------------------------------------------------------------------- 1 | 1.3498805449537964E9 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/theta_multiple_join.result: -------------------------------------------------------------------------------- 1 | 1504763.4367 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/theta_output_dominated.result: -------------------------------------------------------------------------------- 1 | 1.25125E7 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/theta_tpch7.result: -------------------------------------------------------------------------------- 1 | GERMANY|FRANCE|1995 = 6232818.703699999 2 | GERMANY|FRANCE|1996 = 5557312.1121000005 3 | FRANCE|GERMANY|1995 = 4637235.1501 4 | FRANCE|GERMANY|1996 = 5224779.573599998 5 | -------------------------------------------------------------------------------- /test/results/tpch/0.1G/tpch19.result: -------------------------------------------------------------------------------- 1 | 2.0312610470745897E9 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/tpch4.result: -------------------------------------------------------------------------------- 1 | 2-HIGH = 997 2 | 5-LOW = 1077 3 | 3-MEDIUM = 1031 4 | 4-NOT SPECIFIED = 989 5 | 1-URGENT = 999 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/tpch5.result: -------------------------------------------------------------------------------- 1 | INDONESIA = 5580475.4026999995 2 | VIETNAM = 4497840.546600002 3 | CHINA = 7822102.999999997 4 | JAPAN = 6000077.218400001 5 | INDIA = 6376121.5084999995 -------------------------------------------------------------------------------- /test/results/tpch/0.1G/tpch7.result: -------------------------------------------------------------------------------- 1 | GERMANY|FRANCE|1995 = 6232818.703699999 2 | GERMANY|FRANCE|1996 = 5557312.1121000005 3 | FRANCE|GERMANY|1995 = 4637235.1501 4 | FRANCE|GERMANY|1996 = 5224779.573599998 -------------------------------------------------------------------------------- /test/squall/schemas/Ex1.txt: -------------------------------------------------------------------------------- 1 | TABLE typesOnly1 [100:S] ( 2 | c11 LONG , 3 | c12 DOUBLE , 4 | C13 STRING , 5 | c14 DATE 6 | ); 7 | 8 | TABLE distinct2 [200] ( 9 | c21 LONG DISTINCT=21 , 10 | c22 DOUBLE DISTINCT=22 , 11 | c23 STRING DISTINCT=23 , 12 | c24 DATE DISTINCT=24 13 | ); 14 | 15 | TABLE min3 [300] ( 16 | c31 LONG MIN=31 , 17 | c32 DOUBLE MIN=32.32 , 18 | c33 STRING MIN=33A , 19 | c34 DATE MIN=1983-11-28 20 | ); 21 | 22 | TABLE max4 ( 23 | c41 LONG MAX=41 , 24 | c42 DOUBLE MAX=42.42 , 25 | c43 STRING MAX=43A , 26 | c44 DATE MAX=1984-12-23 27 | ); 28 | -------------------------------------------------------------------------------- /test/squall/schemas/Ex2.txt: -------------------------------------------------------------------------------- 1 | TABLE orderDifferent1 [100:S] ( 2 | c11 LONG DISTINCT=11 MAX=111 MIN=1 , 3 | c12 DOUBLE DISTINCT=12 MAX=121.21 MIN=1.2 , 4 | C13 STRING DISTINCT=133:S , 5 | c14 DATE MAX=2015-12-23 MIN=1985-11-28 DISTINCT=144 6 | ); -------------------------------------------------------------------------------- /test/squall/schemas/rst.txt: -------------------------------------------------------------------------------- 1 | TABLE R [10000] ( 2 | A LONG, 3 | B LONG, 4 | ); 5 | 6 | TABLE S [10000] ( 7 | B LONG, 8 | C LONG, 9 | ); 10 | 11 | TABLE T [10000] ( 12 | C LONG, 13 | D LONG, 14 | ); -------------------------------------------------------------------------------- /test/squall/sql_queries/distinct_hyracks.sql: -------------------------------------------------------------------------------- 1 | #Hyracks:ver1.0 2 | 3 | SELECT CUSTOMER.MKTSEGMENT, COUNT(DISTINCT ORDERS.ORDERKEY) 4 | FROM CUSTOMER join ORDERS on CUSTOMER.CUSTKEY=ORDERS.CUSTKEY 5 | GROUP BY CUSTOMER.MKTSEGMENT 6 | -------------------------------------------------------------------------------- /test/squall/sql_queries/hyracks.sql: -------------------------------------------------------------------------------- 1 | #Hyracks:ver1.0 2 | 3 | SELECT CUSTOMER.MKTSEGMENT, COUNT(ORDERS.ORDERKEY) 4 | FROM CUSTOMER join ORDERS on CUSTOMER.CUSTKEY=ORDERS.CUSTKEY 5 | GROUP BY CUSTOMER.MKTSEGMENT 6 | -------------------------------------------------------------------------------- /test/squall/sql_queries/tpch10.sql: -------------------------------------------------------------------------------- 1 | #TPCH10:ver1.0 2 | 3 | SELECT CUSTOMER.CUSTKEY, CUSTOMER.NAME, SUM(LINEITEM.EXTENDEDPRICE * (1.0 - LINEITEM.DISCOUNT)), CUSTOMER.ACCTBAL, NATION.NAME, CUSTOMER.ADDRESS, CUSTOMER.PHONE, CUSTOMER.COMMENT FROM CUSTOMER 4 | inner join ORDERS on CUSTOMER.CUSTKEY = ORDERS.CUSTKEY 5 | inner join NATION on NATION.NATIONKEY = CUSTOMER.NATIONKEY 6 | inner join LINEITEM on LINEITEM.ORDERKEY = ORDERS.ORDERKEY 7 | WHERE ORDERS.ORDERDATE >= {d '1993-10-01'} AND ORDERS.ORDERDATE < {d '1994-01-01'} 8 | AND LINEITEM.RETURNFLAG = 'R' 9 | GROUP BY CUSTOMER.CUSTKEY, CUSTOMER.NAME, CUSTOMER.ACCTBAL, NATION.NAME, CUSTOMER.ADDRESS, CUSTOMER.PHONE, CUSTOMER.COMMENT 10 | -------------------------------------------------------------------------------- /test/squall/sql_queries/tpch12.sql: -------------------------------------------------------------------------------- 1 | SELECT LINEITEM.SHIPMODE, ORDERS.ORDERPRIORITY, COUNT(ORDERS.ORDERPRIORITY) 2 | FROM ORDERS inner join LINEITEM on 3 | ORDERS.ORDERKEY = LINEITEM.ORDERKEY 4 | WHERE (ORDERS.ORDERPRIORITY = '1-URGENT' OR ORDERS.ORDERPRIORITY = '2-HIGH' 5 | OR ORDERS.ORDERPRIORITY = '3-MEDIUM' OR ORDERS.ORDERPRIORITY = '4-NOT SPECIFIED' 6 | or ORDERS.ORDERPRIORITY = '5-LOW' 7 | ) 8 | AND (LINEITEM.SHIPMODE = 'MAIL' OR LINEITEM.SHIPMODE ='SHIP') 9 | AND LINEITEM.COMMITDATE < LINEITEM.RECEIPTDATE 10 | AND LINEITEM.SHIPDATE < LINEITEM.COMMITDATE 11 | AND LINEITEM.RECEIPTDATE >= {d '1994-01-01'} 12 | AND LINEITEM.RECEIPTDATE < {d '1995-10-01'} 13 | GROUP BY LINEITEM.SHIPMODE, ORDERS.ORDERPRIORITY 14 | -------------------------------------------------------------------------------- /test/squall/sql_queries/tpch19.sql: -------------------------------------------------------------------------------- 1 | SELECT SUM(LINEITEM.EXTENDEDPRICE * (1.0 - LINEITEM.DISCOUNT)) 2 | FROM LINEITEM INNER JOIN PART ON LINEITEM.PARTKEY = PART.PARTKEY 3 | WHERE (PART.CONTAINER = 'SM CASE') OR (PART.CONTAINER = 'SM BOX') OR 4 | (PART.CONTAINER = 'SM PACK') OR (PART.CONTAINER ='SM PKG') 5 | -------------------------------------------------------------------------------- /test/squall/sql_queries/tpch3.sql: -------------------------------------------------------------------------------- 1 | # TPCH3:ver1.0 2 | 3 | SELECT LINEITEM.ORDERKEY, SUM(LINEITEM.EXTENDEDPRICE*(1.0-LINEITEM.DISCOUNT)), ORDERS.ORDERDATE, ORDERS.SHIPPRIORITY 4 | FROM CUSTOMER inner join ORDERS on CUSTOMER.CUSTKEY = ORDERS.CUSTKEY 5 | inner join LINEITEM on LINEITEM.ORDERKEY = ORDERS.ORDERKEY 6 | WHERE CUSTOMER.MKTSEGMENT = 'BUILDING' and 7 | ORDERS.ORDERDATE < {d '1995-03-15'} and LINEITEM.SHIPDATE > {d '1995-03-15'} 8 | GROUP BY LINEITEM.ORDERKEY, ORDERS.ORDERDATE, ORDERS.SHIPPRIORITY 9 | -------------------------------------------------------------------------------- /test/squall/sql_queries/tpch4.sql: -------------------------------------------------------------------------------- 1 | SELECT ORDERS.ORDERPRIORITY, COUNT(DISTINCT(LINEITEM.ORDERKEY)) 2 | FROM LINEITEM inner join ORDERS on LINEITEM.ORDERKEY = ORDERS.ORDERKEY 3 | WHERE ORDERS.ORDERDATE >= {d '1993-07-01'} AND ORDERS.ORDERDATE < {d '1993-10-01'} 4 | AND (LINEITEM.COMMITDATE < LINEITEM.RECEIPTDATE) 5 | -------------------------------------------------------------------------------- /test/squall/sql_queries/tpch5.sql: -------------------------------------------------------------------------------- 1 | # TPCH5:ver1.0 2 | 3 | SELECT NATION.NAME, SUM(LINEITEM.EXTENDEDPRICE * (1.0 - LINEITEM.DISCOUNT)) 4 | FROM CUSTOMER inner join ORDERS on CUSTOMER.CUSTKEY = ORDERS.CUSTKEY 5 | inner join SUPPLIER on CUSTOMER.NATIONKEY = SUPPLIER.NATIONKEY 6 | inner join LINEITEM on LINEITEM.ORDERKEY = ORDERS.ORDERKEY AND LINEITEM.SUPPKEY = SUPPLIER.SUPPKEY 7 | inner join NATION on SUPPLIER.NATIONKEY = NATION.NATIONKEY 8 | inner join REGION on NATION.REGIONKEY = REGION.REGIONKEY 9 | WHERE REGION.NAME = 'ASIA' AND 10 | ORDERS.ORDERDATE >= {d '1994-01-01'} AND ORDERS.ORDERDATE < {d '1995-01-01'} 11 | GROUP BY NATION.NAME 12 | -------------------------------------------------------------------------------- /test/squall/sql_queries/tpch6.sql: -------------------------------------------------------------------------------- 1 | SELECT SUM(LINEITEM.EXTENDEDPRICE*LINEITEM.DISCOUNT) 2 | FROM LINEITEM 3 | WHERE LINEITEM.SHIPDATE >= {d '1994-01-01'} AND LINEITEM.SHIPDATE < {d '1995-01-01'} 4 | AND LINEITEM.DISCOUNT >= 0.05 AND LINEITEM.DISCOUNT <= 0.07 AND LINEITEM.QUANTITY < 24 5 | -------------------------------------------------------------------------------- /test/squall/sql_queries/tpch7.sql: -------------------------------------------------------------------------------- 1 | #TPCH7:ver1.0 2 | 3 | SELECT N1.NAME, N2.NAME, EXTRACT_YEAR(LINEITEM.SHIPDATE), SUM(LINEITEM.EXTENDEDPRICE * (1.0-LINEITEM.DISCOUNT)) 4 | FROM SUPPLIER 5 | inner join LINEITEM on SUPPLIER.SUPPKEY = LINEITEM.SUPPKEY 6 | inner join NATION N1 on SUPPLIER.NATIONKEY = N1.NATIONKEY 7 | inner join ORDERS on ORDERS.ORDERKEY = LINEITEM.ORDERKEY 8 | inner join CUSTOMER on CUSTOMER.CUSTKEY = ORDERS.CUSTKEY 9 | inner join NATION N2 on CUSTOMER.NATIONKEY = N2.NATIONKEY 10 | WHERE ((N1.NAME = 'FRANCE' AND N2.NAME = 'GERMANY') OR (N1.NAME = 'GERMANY' AND N2.NAME = 'FRANCE')) 11 | AND LINEITEM.SHIPDATE >= {d '1995-01-01'} AND LINEITEM.SHIPDATE <= {d '1996-12-31'} 12 | -------------------------------------------------------------------------------- /test/squall/sql_queries/tpch8.sql: -------------------------------------------------------------------------------- 1 | #TPCH8:ver1.0 2 | 3 | SELECT N2.NAME, EXTRACT_YEAR(ORDERS.ORDERDATE), SUM(LINEITEM.EXTENDEDPRICE*(1.0-LINEITEM.DISCOUNT)) FROM PART 4 | inner join LINEITEM on PART.PARTKEY = LINEITEM.PARTKEY 5 | inner join SUPPLIER on SUPPLIER.SUPPKEY = LINEITEM.SUPPKEY 6 | inner join NATION N2 on SUPPLIER.NATIONKEY = N2.NATIONKEY 7 | inner join ORDERS on LINEITEM.ORDERKEY = ORDERS.ORDERKEY 8 | inner join CUSTOMER on ORDERS.CUSTKEY = CUSTOMER.CUSTKEY 9 | inner join NATION N1 on CUSTOMER.NATIONKEY = N1.NATIONKEY 10 | inner join REGION on N1.REGIONKEY = REGION.REGIONKEY 11 | WHERE REGION.NAME = 'AMERICA' AND PART.TYPE = 'ECONOMY ANODIZED STEEL' AND 12 | ORDERS.ORDERDATE >= {d '1995-01-01'} AND ORDERS.ORDERDATE <= {d '1996-12-31'} 13 | GROUP BY N2.NAME, ORDERS.ORDERDATE 14 | -------------------------------------------------------------------------------- /test/squall/sql_queries/tpch9.sql: -------------------------------------------------------------------------------- 1 | #TPCH9:ver1.0 2 | 3 | SELECT NATION.NAME, EXTRACT_YEAR(ORDERS.ORDERDATE), SUM((LINEITEM.EXTENDEDPRICE*(1.0-LINEITEM.DISCOUNT))-(PARTSUPP.SUPPLYCOST * LINEITEM.QUANTITY)) FROM PART 4 | inner join LINEITEM on PART.PARTKEY = LINEITEM.PARTKEY 5 | inner join PARTSUPP on PARTSUPP.PARTKEY = LINEITEM.PARTKEY AND PARTSUPP.SUPPKEY = LINEITEM.SUPPKEY 6 | inner join ORDERS on LINEITEM.ORDERKEY = ORDERS.ORDERKEY 7 | inner join SUPPLIER on SUPPLIER.SUPPKEY = LINEITEM.SUPPKEY 8 | inner join NATION on NATION.NATIONKEY = SUPPLIER.NATIONKEY 9 | WHERE PART.NAME LIKE '%green%' 10 | GROUP BY NATION.NAME, ORDERS.ORDERDATE 11 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hash_hypercube_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.hypercube.HashHypercubeDBToasterHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z0/40G/ 8 | 9 | CUSTOMER_PAR 3 10 | ORDERS_PAR 8 11 | CUSTOMER_ORDERS_PAR 16 12 | 13 | COUNTAGG_PAR 1 14 | SENDRESULTSTOREDIS_PAR 1 15 | 16 | CUSTOMER_CARD 1500 17 | ORDERS_CARD 15000 18 | 19 | CUSTOMER_ORDERS_PART_SCHEME HASHHYPERCUBE 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .tbl 23 | DIP_READ_SPLIT_DELIMITER \| 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 36 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hash_hypercube_tpch10: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch10 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterTPCH10Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/80G/ 8 | 9 | CUSTOMER_PAR 4 10 | ORDERS_PAR 5 11 | NATION_PAR 1 12 | LINEITEM_PAR 8 13 | 14 | CUSTOMER_ORDERS_NATION_LINEITEM_PAR 64 15 | SUMAGG_PAR 8 16 | 17 | CUSTOMER_ORDERS_NATION_LINEITEM_PART_SCHEME HASHHYPERCUBE 18 | 19 | CUSTOMER_CARD 1500 20 | ORDERS_CARD 15000 21 | LINEITEM_CARD 60175 22 | NATION_CARD 25 23 | 24 | DIP_INPUT_FREQ_PRINT 200000 25 | 26 | #below are unlikely to change 27 | DIP_EXTENSION .tbl 28 | DIP_READ_SPLIT_DELIMITER \| 29 | DIP_GLOBAL_ADD_DELIMITER | 30 | DIP_GLOBAL_SPLIT_DELIMITER \| 31 | 32 | DIP_KILL_AT_THE_END true 33 | 34 | # Storage manager parameters 35 | # Storage directory for local runs 36 | STORAGE_LOCAL_DIR /tmp/ramdisk 37 | # Storage directory for cluster runs 38 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 39 | STORAGE_COLD_START true 40 | STORAGE_MEMORY_SIZE_MB 4096 41 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hash_hypercube_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.HashHypercubeDBToasterTPCH3Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | CUSTOMER_PAR 1 10 | ORDERS_PAR 2 11 | LINEITEM_PAR 3 12 | CUSTOMER_ORDERS_LINEITEM_PAR 8 13 | COUNTAGG_PAR 1 14 | 15 | CUSTOMER_CARD 1500 16 | ORDERS_CARD 15000 17 | LINEITEM_CARD 60175 18 | 19 | CUSTOMER_ORDERS_LINEITEM_PART_SCHEME HASHHYPERCUBE 20 | 21 | DIP_INPUT_FREQ_PRINT 200000 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 38 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hash_hypercube_tpch5: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch5 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.HashHypercubeDBToasterTPCH5Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | REGION_PAR 1 10 | NATION_PAR 1 11 | SUPPLIER_PAR 1 12 | LINEITEM_PAR 4 13 | CUSTOMER_PAR 2 14 | ORDERS_PAR 3 15 | 16 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PAR 64 17 | 18 | FINAL_RESULT_PAR 1 19 | 20 | CUSTOMER_CARD 1500 21 | ORDERS_CARD 15000 22 | LINEITEM_CARD 60175 23 | REGION_CARD 5 24 | NATION_CARD 25 25 | SUPPLIER_CARD 100 26 | 27 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PART_SCHEME HASHHYPERCUBE 28 | 29 | DIP_INPUT_FREQ_PRINT 200000 30 | 31 | #below are unlikely to change 32 | DIP_EXTENSION .tbl 33 | DIP_READ_SPLIT_DELIMITER \| 34 | DIP_GLOBAL_ADD_DELIMITER | 35 | DIP_GLOBAL_SPLIT_DELIMITER \| 36 | 37 | DIP_KILL_AT_THE_END true 38 | 39 | # Storage manager parameters 40 | # Storage directory for local runs 41 | STORAGE_LOCAL_DIR /tmp/ramdisk 42 | # Storage directory for cluster runs 43 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 44 | STORAGE_COLD_START true 45 | STORAGE_MEMORY_SIZE_MB 4096 46 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hash_hypercube_tpch9: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch9 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.hypercube.HashHypercubeDBToasterTPCH9Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | PART_PAR 1 10 | LINEITEM_PAR 8 11 | PARTSUPP_PAR 4 12 | ORDERS_PAR 4 13 | SUPPLIER_PAR 1 14 | NATION_PAR 1 15 | 16 | PART_LINEITEM_PARTSUPP_ORDERS_SUPPLIER_NATION_PAR 77 17 | 18 | PART_LINEITEM_PARTSUPP_ORDERS_SUPPLIER_NATION_PART_SCHEME HASHHYPERCUBE 19 | 20 | ORDERS_CARD 15000 21 | LINEITEM_CARD 60175 22 | REGION_CARD 5 23 | NATION_CARD 25 24 | SUPPLIER_CARD 100 25 | PART_CARD 50 26 | PARTSUPP_CARD 50 27 | 28 | DIP_INPUT_FREQ_PRINT 200000 29 | 30 | #below are unlikely to change 31 | DIP_EXTENSION .tbl 32 | DIP_READ_SPLIT_DELIMITER \| 33 | DIP_GLOBAL_ADD_DELIMITER | 34 | DIP_GLOBAL_SPLIT_DELIMITER \| 35 | 36 | DIP_KILL_AT_THE_END true 37 | 38 | # Storage manager parameters 39 | # Storage directory for local runs 40 | STORAGE_LOCAL_DIR /tmp/ramdisk 41 | # Storage directory for cluster runs 42 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 43 | STORAGE_COLD_START true 44 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hash_hypercube_tpch9_partial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch9 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.HashHypercubeDBToasterTPCH9PartialPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/80G/ 8 | 9 | #PART_PAR 1 10 | #LINEITEM_PAR 4 11 | #PARTSUPP_PAR 1 12 | #PART_LINEITEM_PARTSUPP_PAR 8 13 | 14 | PART_PAR 1 15 | LINEITEM_PAR 10 16 | PARTSUPP_PAR 4 17 | PART_LINEITEM_PARTSUPP_PAR 100 18 | 19 | PART_LINEITEM_PARTSUPP_PART_SCHEME HASHHYPERCUBE 20 | 21 | LINEITEM_CARD 59986060 22 | PART_CARD 106280 23 | PARTSUPP_CARD 8000000 24 | 25 | DIP_INPUT_FREQ_PRINT 200000 26 | 27 | #below are unlikely to change 28 | DIP_EXTENSION .tbl 29 | DIP_READ_SPLIT_DELIMITER \| 30 | DIP_GLOBAL_ADD_DELIMITER | 31 | DIP_GLOBAL_SPLIT_DELIMITER \| 32 | 33 | DIP_KILL_AT_THE_END true 34 | 35 | # Storage manager parameters 36 | # Storage directory for local runs 37 | STORAGE_LOCAL_DIR /tmp/ramdisk 38 | # Storage directory for cluster runs 39 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 40 | STORAGE_COLD_START true 41 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hybrid_hypercube_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.hypercube.HybridHypercubeDBToasterHyracksPlanBruteForce 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | CUSTOMER_PAR 1 10 | ORDERS_PAR 2 11 | CUSTOMER_ORDERS_PAR 8 12 | 13 | CUSTOMER_CARD 1500 14 | ORDERS_CARD 15000 15 | 16 | CUSTOMER_ORDERS_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 17 | 18 | DIP_INPUT_FREQ_PRINT 200000 19 | 20 | #below are unlikely to change 21 | DIP_EXTENSION .tbl 22 | DIP_READ_SPLIT_DELIMITER \| 23 | DIP_GLOBAL_ADD_DELIMITER | 24 | DIP_GLOBAL_SPLIT_DELIMITER \| 25 | 26 | DIP_KILL_AT_THE_END true 27 | 28 | # Storage manager parameters 29 | # Storage directory for local runs 30 | STORAGE_LOCAL_DIR /tmp/ramdisk 31 | # Storage directory for cluster runs 32 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 33 | STORAGE_COLD_START true 34 | STORAGE_MEMORY_SIZE_MB 4096 35 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hybrid_hypercube_manual_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.hypercube.HybridHypercubeDBToasterHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z1/10G/ 8 | 9 | CUSTOMER_PAR 8 10 | ORDERS_PAR 8 11 | CUSTOMER_ORDERS_PAR 16 12 | COUNTAGG_PAR 1 13 | 14 | CUSTOMER_CARD 50 15 | ORDERS_CARD 50 16 | 17 | # CUSTOMER_ORDERS_PAR should be equal CUSTOMER_DIMENSION * column1_DIMENSION 18 | CUSTOMER_DIMENSION 4 19 | column1_DIMENSION 4 20 | CUSTOMER_ORDERS_PART_SCHEME MANUALHYBRIDHYPERCUBE 21 | 22 | DIP_OUTPUT_FREQ_PRINT 200000 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .tbl 26 | DIP_READ_SPLIT_DELIMITER \| 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hybrid_hypercube_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.HybridHypercubeDBToasterTPCH3PlanBruteForce 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | CUSTOMER_PAR 1 10 | ORDERS_PAR 2 11 | LINEITEM_PAR 3 12 | CUSTOMER_ORDERS_LINEITEM_PAR 8 13 | COUNTAGG_PAR 1 14 | 15 | CUSTOMER_CARD 1500 16 | ORDERS_CARD 15000 17 | LINEITEM_CARD 60175 18 | 19 | CUSTOMER_ORDERS_LINEITEM_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 20 | 21 | DIP_INPUT_FREQ_PRINT 200000 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 38 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hybrid_hypercube_tpch5: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch5 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.HybridHypercubeDBToasterTPCH5Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z0/10G/ 8 | 9 | REGION_PAR 1 10 | NATION_PAR 1 11 | SUPPLIER_PAR 1 12 | LINEITEM_PAR 8 13 | CUSTOMER_PAR 4 14 | ORDERS_PAR 5 15 | 16 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PAR 64 17 | 18 | FINAL_RESULT_PAR 1 19 | 20 | CUSTOMER_CARD 1500 21 | ORDERS_CARD 15000 22 | LINEITEM_CARD 60175 23 | REGION_CARD 5 24 | NATION_CARD 25 25 | SUPPLIER_CARD 100 26 | 27 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 28 | 29 | DIP_INPUT_FREQ_PRINT 200000 30 | 31 | #below are unlikely to change 32 | DIP_EXTENSION .tbl 33 | DIP_READ_SPLIT_DELIMITER \| 34 | DIP_GLOBAL_ADD_DELIMITER | 35 | DIP_GLOBAL_SPLIT_DELIMITER \| 36 | 37 | DIP_KILL_AT_THE_END true 38 | 39 | # Storage manager parameters 40 | # Storage directory for local runs 41 | STORAGE_LOCAL_DIR /tmp/ramdisk 42 | # Storage directory for cluster runs 43 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 44 | STORAGE_COLD_START true 45 | STORAGE_MEMORY_SIZE_MB 4096 46 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hybrid_hypercube_tpch9: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch9 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.hypercube.HybridHypercubeDBToasterTPCH9Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | PART_PAR 1 10 | LINEITEM_PAR 8 11 | PARTSUPP_PAR 4 12 | ORDERS_PAR 4 13 | SUPPLIER_PAR 1 14 | NATION_PAR 1 15 | 16 | PART_LINEITEM_PARTSUPP_ORDERS_SUPPLIER_NATION_PAR 77 17 | 18 | PART_LINEITEM_PARTSUPP_ORDERS_SUPPLIER_NATION_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 19 | 20 | ORDERS_CARD 15000 21 | LINEITEM_CARD 60175 22 | REGION_CARD 5 23 | NATION_CARD 25 24 | SUPPLIER_CARD 100 25 | PART_CARD 50 26 | PARTSUPP_CARD 50 27 | 28 | DIP_INPUT_FREQ_PRINT 200000 29 | 30 | #below are unlikely to change 31 | DIP_EXTENSION .tbl 32 | DIP_READ_SPLIT_DELIMITER \| 33 | DIP_GLOBAL_ADD_DELIMITER | 34 | DIP_GLOBAL_SPLIT_DELIMITER \| 35 | 36 | DIP_KILL_AT_THE_END true 37 | 38 | # Storage manager parameters 39 | # Storage directory for local runs 40 | STORAGE_LOCAL_DIR /tmp/ramdisk 41 | # Storage directory for cluster runs 42 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 43 | STORAGE_COLD_START true 44 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hybrid_hypercube_tpch9_partial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch9 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.HybridHypercubeDBToasterTPCH9PartialPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/80G/ 8 | 9 | #PART_PAR 1 10 | #LINEITEM_PAR 4 11 | #PARTSUPP_PAR 1 12 | #PART_LINEITEM_PARTSUPP_PAR 8 13 | 14 | PART_PAR 1 15 | LINEITEM_PAR 12 16 | PARTSUPP_PAR 4 17 | PART_LINEITEM_PARTSUPP_PAR 96 18 | 19 | #PART_PAR 1 20 | #LINEITEM_PAR 12 21 | #PARTSUPP_PAR 4 22 | #PART_LINEITEM_PARTSUPP_PAR 88 23 | 24 | PART_LINEITEM_PARTSUPP_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 25 | 26 | LINEITEM_CARD 59986060 27 | PART_CARD 106280 28 | PARTSUPP_CARD 8000000 29 | 30 | DIP_INPUT_FREQ_PRINT 200000 31 | 32 | #below are unlikely to change 33 | DIP_EXTENSION .tbl 34 | DIP_READ_SPLIT_DELIMITER \| 35 | DIP_GLOBAL_ADD_DELIMITER | 36 | DIP_GLOBAL_SPLIT_DELIMITER \| 37 | 38 | DIP_KILL_AT_THE_END true 39 | 40 | # Storage manager parameters 41 | # Storage directory for local runs 42 | STORAGE_LOCAL_DIR /tmp/ramdisk 43 | # Storage directory for cluster runs 44 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 45 | STORAGE_COLD_START true 46 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z0/40G/ 8 | 9 | CUSTOMER_PAR 3 10 | ORDERS_PAR 4 11 | CUSTOMER_ORDERS_PAR 16 12 | COUNTAGG_PAR 4 13 | 14 | CUSTOMER_CARD 1500 15 | ORDERS_CARD 15000 16 | 17 | DIP_INPUT_FREQ_PRINT 200000 18 | 19 | #below are unlikely to change 20 | DIP_EXTENSION .tbl 21 | DIP_READ_SPLIT_DELIMITER \| 22 | DIP_GLOBAL_ADD_DELIMITER | 23 | DIP_GLOBAL_SPLIT_DELIMITER \| 24 | 25 | DIP_KILL_AT_THE_END true 26 | 27 | # Storage manager parameters 28 | # Storage directory for local runs 29 | STORAGE_LOCAL_DIR /tmp/ramdisk 30 | # Storage directory for cluster runs 31 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 32 | STORAGE_COLD_START true 33 | STORAGE_MEMORY_SIZE_MB 4096 34 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_seqjoin_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterTPCH3SequentialJoinPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | CUSTOMER_PAR 4 10 | ORDERS_PAR 4 11 | LINEITEM_PAR 4 12 | CUSTOMER_ORDERS_PAR 2 13 | CUSTOMER_ORDERS_LINEITEM_PAR 4 14 | COUNTAGG_PAR 1 15 | 16 | CUSTOMER_CARD 1500 17 | ORDERS_CARD 15000 18 | CUSTOMER_ORDERS_CARD 1500 19 | LINEITEM_CARD 60175 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .tbl 23 | DIP_READ_SPLIT_DELIMITER \| 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 36 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_seqjoin_tpch3_hash: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterTPCH3SequentialJoinPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | CUSTOMER_PAR 4 10 | ORDERS_PAR 4 11 | LINEITEM_PAR 2 12 | CUSTOMER_ORDERS_PAR 2 13 | CUSTOMER_ORDERS_LINEITEM_PAR 4 14 | COUNTAGG_PAR 1 15 | 16 | CUSTOMER_ORDERS_PART_SCHEME HASH 17 | CUSTOMER_ORDERS_LINEITEM_PART_SCHEME HASH 18 | 19 | CUSTOMER_CARD 1500 20 | ORDERS_CARD 15000 21 | CUSTOMER_ORDERS_CARD 1500 22 | LINEITEM_CARD 60175 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .tbl 26 | DIP_READ_SPLIT_DELIMITER \| 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_tpch10: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch10 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterTPCH10Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | CUSTOMER_PAR 4 10 | ORDERS_PAR 5 11 | NATION_PAR 1 12 | LINEITEM_PAR 8 13 | 14 | CUSTOMER_ORDERS_NATION_LINEITEM_PAR 64 15 | SUMAGG_PAR 1 16 | 17 | CUSTOMER_CARD 1500 18 | ORDERS_CARD 15000 19 | LINEITEM_CARD 60175 20 | NATION_CARD 25 21 | 22 | DIP_INPUT_FREQ_PRINT 200000 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .tbl 26 | DIP_READ_SPLIT_DELIMITER \| 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterTPCH3Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | CUSTOMER_PAR 1 10 | ORDERS_PAR 2 11 | LINEITEM_PAR 3 12 | CUSTOMER_ORDERS_LINEITEM_PAR 8 13 | COUNTAGG_PAR 1 14 | 15 | DIP_INPUT_FREQ_PRINT 200000 16 | 17 | CUSTOMER_CARD 1500 18 | ORDERS_CARD 15000 19 | LINEITEM_CARD 60175 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .tbl 23 | DIP_READ_SPLIT_DELIMITER \| 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 36 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_tpch5: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch5 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterTPCH5Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | REGION_PAR 1 10 | NATION_PAR 1 11 | SUPPLIER_PAR 1 12 | LINEITEM_PAR 8 13 | CUSTOMER_PAR 4 14 | ORDERS_PAR 5 15 | 16 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PAR 64 17 | FINAL_RESULT_PAR 1 18 | 19 | CUSTOMER_CARD 1500 20 | ORDERS_CARD 15000 21 | LINEITEM_CARD 60175 22 | REGION_CARD 5 23 | NATION_CARD 25 24 | SUPPLIER_CARD 100 25 | 26 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PART_SCHEME HYPERCUBE 27 | 28 | DIP_INPUT_FREQ_PRINT 200000 29 | 30 | #below are unlikely to change 31 | DIP_EXTENSION .tbl 32 | DIP_READ_SPLIT_DELIMITER \| 33 | DIP_GLOBAL_ADD_DELIMITER | 34 | DIP_GLOBAL_SPLIT_DELIMITER \| 35 | 36 | DIP_KILL_AT_THE_END true 37 | 38 | # Storage manager parameters 39 | # Storage directory for local runs 40 | STORAGE_LOCAL_DIR /tmp/ramdisk 41 | # Storage directory for cluster runs 42 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 43 | STORAGE_COLD_START true 44 | STORAGE_MEMORY_SIZE_MB 4096 45 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_tpch9: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch9 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterTPCH9Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | PART_PAR 1 10 | LINEITEM_PAR 8 11 | PARTSUPP_PAR 4 12 | ORDERS_PAR 4 13 | SUPPLIER_PAR 1 14 | NATION_PAR 1 15 | 16 | PART_LINEITEM_PARTSUPP_ORDERS_SUPPLIER_NATION_PAR 77 17 | 18 | ORDERS_CARD 15000 19 | LINEITEM_CARD 60175 20 | REGION_CARD 5 21 | NATION_CARD 25 22 | SUPPLIER_CARD 100 23 | PART_CARD 50 24 | PARTSUPP_CARD 50 25 | 26 | DIP_INPUT_FREQ_PRINT 200000 27 | 28 | #below are unlikely to change 29 | DIP_EXTENSION .tbl 30 | DIP_READ_SPLIT_DELIMITER \| 31 | DIP_GLOBAL_ADD_DELIMITER | 32 | DIP_GLOBAL_SPLIT_DELIMITER \| 33 | 34 | DIP_KILL_AT_THE_END true 35 | 36 | # Storage manager parameters 37 | # Storage directory for local runs 38 | STORAGE_LOCAL_DIR /tmp/ramdisk 39 | # Storage directory for cluster runs 40 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 41 | STORAGE_COLD_START true 42 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_tpch9_partial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch9 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterTPCH9PartialPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/80G/ 8 | 9 | PART_PAR 1 10 | LINEITEM_PAR 12 11 | PARTSUPP_PAR 4 12 | PART_LINEITEM_PARTSUPP_PAR 100 13 | 14 | LINEITEM_CARD 59986060 15 | PART_CARD 106280 16 | PARTSUPP_CARD 8000000 17 | 18 | #PART_PAR 1 19 | #LINEITEM_PAR 16 20 | #PARTSUPP_PAR 4 21 | #PART_LINEITEM_PARTSUPP_PAR 64 22 | 23 | #PART_PAR 1 24 | #LINEITEM_PAR 4 25 | #PARTSUPP_PAR 1 26 | 27 | #PART_LINEITEM_PARTSUPP_PAR 8 28 | 29 | #LINEITEM_CARD 59986060 30 | #PART_CARD 106280 31 | #PARTSUPP_CARD 8000000 32 | 33 | DIP_INPUT_FREQ_PRINT 200000 34 | 35 | #below are unlikely to change 36 | DIP_EXTENSION .tbl 37 | DIP_READ_SPLIT_DELIMITER \| 38 | DIP_GLOBAL_ADD_DELIMITER | 39 | DIP_GLOBAL_SPLIT_DELIMITER \| 40 | 41 | DIP_KILL_AT_THE_END true 42 | 43 | # Storage manager parameters 44 | # Storage directory for local runs 45 | STORAGE_LOCAL_DIR /tmp/ramdisk 46 | # Storage directory for cluster runs 47 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 48 | STORAGE_COLD_START true 49 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_dbtoaster_z1_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z1/10G/ 8 | 9 | CUSTOMER_PAR 8 10 | ORDERS_PAR 8 11 | CUSTOMER_ORDERS_PAR 16 12 | COUNTAGG_PAR 16 13 | 14 | CUSTOMER_CARD 50 15 | ORDERS_CARD 50 16 | 17 | DIP_OUTPUT_FREQ_PRINT 200000 18 | 19 | #below are unlikely to change 20 | DIP_EXTENSION .tbl 21 | DIP_READ_SPLIT_DELIMITER \| 22 | DIP_GLOBAL_ADD_DELIMITER | 23 | DIP_GLOBAL_SPLIT_DELIMITER \| 24 | 25 | DIP_KILL_AT_THE_END true 26 | 27 | # Storage manager parameters 28 | # Storage directory for local runs 29 | STORAGE_LOCAL_DIR /tmp/ramdisk 30 | # Storage directory for cluster runs 31 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 32 | STORAGE_COLD_START true 33 | STORAGE_MEMORY_SIZE_MB 4096 34 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.theta.ThetaHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z0/40G/ 8 | 9 | # THETA JOIN TYPE 10 | # STATIC PARTITIONING = 0 11 | # DYNAMIC PARTITIONING = 1 12 | DIP_JOIN_TYPE 0 13 | 14 | CUSTOMER_PAR 4 15 | ORDERS_PAR 8 16 | 17 | CUSTOMER_ORDERS_PAR 16 18 | COUNTAGG_PAR 16 19 | 20 | CUSTOMER_ORDERS_RESHUF_PAR 4 21 | 22 | CUSTOMER_CARD 1500 23 | ORDERS_CARD 15000 24 | 25 | #below are unlikely to change 26 | DIP_EXTENSION .tbl 27 | DIP_READ_SPLIT_DELIMITER \| 28 | DIP_GLOBAL_ADD_DELIMITER | 29 | DIP_GLOBAL_SPLIT_DELIMITER \| 30 | 31 | DIP_KILL_AT_THE_END true 32 | 33 | # Storage manager parameters 34 | # Storage directory for local runs 35 | STORAGE_LOCAL_DIR /tmp/ramdisk 36 | # Storage directory for cluster runs 37 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 38 | STORAGE_COLD_START true 39 | STORAGE_MEMORY_SIZE_MB 4096 40 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_traditional_hash_hypercube_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.theta.TradionalHypercubeThetaHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | DIP_JOIN_TYPE 0 10 | 11 | CUSTOMER_PAR 1 12 | ORDERS_PAR 2 13 | CUSTOMER_ORDERS_PAR 8 14 | #COUNTAGG_PAR 4 15 | 16 | CUSTOMER_CARD 1500 17 | ORDERS_CARD 15000 18 | 19 | HYPERCUBE_ENABLED true 20 | CUSTOMER_ORDERS_PART_SCHEME HASHHYPERCUBE 21 | 22 | #below are unlikely to change 23 | DIP_EXTENSION .tbl 24 | DIP_READ_SPLIT_DELIMITER \| 25 | DIP_GLOBAL_ADD_DELIMITER | 26 | DIP_GLOBAL_SPLIT_DELIMITER \| 27 | 28 | DIP_KILL_AT_THE_END true 29 | 30 | # Storage manager parameters 31 | # Storage directory for local runs 32 | STORAGE_LOCAL_DIR /tmp/ramdisk 33 | # Storage directory for cluster runs 34 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 35 | STORAGE_COLD_START true 36 | STORAGE_MEMORY_SIZE_MB 4096 37 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_traditional_hash_hypercube_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalHashTPCH3Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | CUSTOMER_PAR 1 10 | ORDERS_PAR 2 11 | LINEITEM_PAR 3 12 | CUSTOMER_ORDERS_LINEITEM_PAR 8 13 | COUNTAGG_PAR 1 14 | 15 | DIP_INPUT_FREQ_PRINT 200000 16 | 17 | CUSTOMER_CARD 1500 18 | ORDERS_CARD 15000 19 | LINEITEM_CARD 60175 20 | 21 | CUSTOMER_ORDERS_LINEITEM_PART_SCHEME HASHHYPERCUBE 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 38 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_traditional_hash_hypercube_tpch9_partial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch9 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalHashTPCH9PartialPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | PART_PAR 1 10 | LINEITEM_PAR 4 11 | PARTSUPP_PAR 1 12 | 13 | PART_LINEITEM_PARTSUPP_PAR 8 14 | 15 | LINEITEM_CARD 60175 16 | PART_CARD 50 17 | PARTSUPP_CARD 50 18 | 19 | DIP_INPUT_FREQ_PRINT 200000 20 | 21 | PART_LINEITEM_PARTSUPP_PART_SCHEME HASHHYPERCUBE 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_traditional_hybrid_hypercube_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.theta.TradionalHypercubeThetaHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | DIP_JOIN_TYPE 0 10 | 11 | CUSTOMER_PAR 2 12 | ORDERS_PAR 4 13 | CUSTOMER_ORDERS_PAR 16 14 | COUNTAGG_PAR 4 15 | 16 | CUSTOMER_CARD 1500 17 | ORDERS_CARD 15000 18 | 19 | HYPERCUBE_ENABLED true 20 | CUSTOMER_ORDERS_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 21 | 22 | #below are unlikely to change 23 | DIP_EXTENSION .tbl 24 | DIP_READ_SPLIT_DELIMITER \| 25 | DIP_GLOBAL_ADD_DELIMITER | 26 | DIP_GLOBAL_SPLIT_DELIMITER \| 27 | 28 | DIP_KILL_AT_THE_END true 29 | 30 | # Storage manager parameters 31 | # Storage directory for local runs 32 | STORAGE_LOCAL_DIR /tmp/ramdisk 33 | # Storage directory for cluster runs 34 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 35 | STORAGE_COLD_START true 36 | STORAGE_MEMORY_SIZE_MB 4096 37 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_traditional_hybrid_hypercube_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalHybridTPCH3Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | CUSTOMER_PAR 1 10 | ORDERS_PAR 2 11 | LINEITEM_PAR 3 12 | CUSTOMER_ORDERS_LINEITEM_PAR 8 13 | COUNTAGG_PAR 1 14 | 15 | DIP_INPUT_FREQ_PRINT 200000 16 | 17 | CUSTOMER_CARD 1500 18 | ORDERS_CARD 15000 19 | LINEITEM_CARD 60175 20 | 21 | CUSTOMER_ORDERS_LINEITEM_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 38 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_traditional_hybrid_hypercube_tpch9_partial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch9 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalHybridTPCH9PartialPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | PART_PAR 1 10 | LINEITEM_PAR 4 11 | PARTSUPP_PAR 1 12 | 13 | PART_LINEITEM_PARTSUPP_PAR 8 14 | 15 | LINEITEM_CARD 60175 16 | PART_CARD 50 17 | PARTSUPP_CARD 50 18 | 19 | DIP_INPUT_FREQ_PRINT 200000 20 | 21 | PART_LINEITEM_PARTSUPP_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_traditional_random_hypercube_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | DIP_JOIN_TYPE 0 10 | 11 | CUSTOMER_PAR 1 12 | ORDERS_PAR 2 13 | CUSTOMER_ORDERS_PAR 8 14 | #COUNTAGG_PAR 4 15 | 16 | CUSTOMER_CARD 1500 17 | ORDERS_CARD 15000 18 | 19 | HYPERCUBE_ENABLED true 20 | CUSTOMER_ORDERS_PART_SCHEME HYPERCUBE 21 | 22 | DIP_INPUT_FREQ_PRINT 200000 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .tbl 26 | DIP_READ_SPLIT_DELIMITER \| 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_traditional_random_hypercube_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalTPCH3Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | CUSTOMER_PAR 1 10 | ORDERS_PAR 2 11 | LINEITEM_PAR 3 12 | CUSTOMER_ORDERS_LINEITEM_PAR 8 13 | COUNTAGG_PAR 1 14 | 15 | DIP_INPUT_FREQ_PRINT 200000 16 | 17 | CUSTOMER_CARD 1500 18 | ORDERS_CARD 15000 19 | LINEITEM_CARD 60175 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .tbl 23 | DIP_READ_SPLIT_DELIMITER \| 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 36 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_traditional_random_hypercube_tpch9_partial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch9 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalHashTPCH9PartialPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | PART_PAR 1 10 | LINEITEM_PAR 4 11 | PARTSUPP_PAR 1 12 | 13 | PART_LINEITEM_PARTSUPP_PAR 8 14 | 15 | LINEITEM_CARD 60175 16 | PART_CARD 50 17 | PARTSUPP_CARD 50 18 | 19 | DIP_INPUT_FREQ_PRINT 200000 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .tbl 23 | DIP_READ_SPLIT_DELIMITER \| 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_traditional_tpch5: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch5 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalTPCH5Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z2/10G/ 8 | 9 | REGION_PAR 1 10 | NATION_PAR 1 11 | SUPPLIER_PAR 1 12 | LINEITEM_PAR 1 13 | CUSTOMER_PAR 1 14 | ORDERS_PAR 1 15 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PAR 128 16 | FINAL_RESULT_PAR 1 17 | 18 | CUSTOMER_CARD 1500 19 | ORDERS_CARD 15000 20 | LINEITEM_CARD 60175 21 | REGION_CARD 5 22 | NATION_CARD 25 23 | SUPPLIER_CARD 100 24 | 25 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PART_SCHEME HYPERCUBE 26 | 27 | DIP_OUTPUT_FREQ_PRINT 200000 28 | 29 | #below are unlikely to change 30 | DIP_EXTENSION .tbl 31 | DIP_READ_SPLIT_DELIMITER \| 32 | DIP_GLOBAL_ADD_DELIMITER | 33 | DIP_GLOBAL_SPLIT_DELIMITER \| 34 | 35 | DIP_KILL_AT_THE_END true 36 | 37 | # Storage manager parameters 38 | # Storage directory for local runs 39 | STORAGE_LOCAL_DIR /tmp/ramdisk 40 | # Storage directory for cluster runs 41 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 42 | STORAGE_COLD_START true 43 | STORAGE_MEMORY_SIZE_MB 4096 44 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_z0_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | #DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.HyracksPlan 4 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.HyracksPredicatePlan 5 | DIP_TOPOLOGY_NAME_PREFIX username 6 | 7 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 8 | # DIP_NUM_WORKERS 2 9 | # DIP_NUM_ACKERS 0 10 | 11 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z0/10G/ 12 | 13 | CUSTOMER_PAR 8 14 | ORDERS_PAR 8 15 | 16 | CUSTOMER_ORDERS_PAR 16 17 | 18 | 19 | DIP_OUTPUT_FREQ_PRINT 200000 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .tbl 23 | DIP_READ_SPLIT_DELIMITER \| 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 36 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_z0_tpch3_4: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch3 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | 5 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 6 | #DIP_NUM_WORKERS 176 7 | #DIP_NUM_ACKERS 0 8 | 9 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z0/10G/ 10 | 11 | CUSTOMER_PAR 4 12 | ORDERS_PAR 4 13 | LINEITEM_PAR 4 14 | 15 | CUSTOMER_ORDERS_PAR 2 16 | CUSTOMER_ORDERS_LINEITEM_PAR 4 17 | 18 | #below are unlikely to change 19 | DIP_EXTENSION .tbl 20 | DIP_READ_SPLIT_DELIMITER \| 21 | DIP_GLOBAL_ADD_DELIMITER | 22 | DIP_GLOBAL_SPLIT_DELIMITER \| 23 | 24 | DIP_KILL_AT_THE_END true 25 | 26 | # Storage manager parameters 27 | # Storage directory for local runs 28 | STORAGE_LOCAL_DIR /tmp/ramdisk 29 | # Storage directory for cluster runs 30 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 31 | STORAGE_COLD_START true 32 | STORAGE_MEMORY_SIZE_MB 4096 33 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_z0_tpch7_4: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch7 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | 5 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 6 | #DIP_NUM_WORKERS 176 7 | #DIP_NUM_ACKERS 0 8 | 9 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z0/10G 10 | 11 | NATION1_PAR 1 12 | NATION2_PAR 1 13 | CUSTOMER_PAR 4 14 | ORDERS_PAR 4 15 | SUPPLIER_PAR 4 16 | LINEITEM_PAR 4 17 | 18 | NATION2_CUSTOMER_PAR 2 19 | NATION2_CUSTOMER_ORDERS_PAR 4 20 | SUPPLIER_NATION1_PAR 2 21 | LINEITEM_SUPPLIER_NATION1_PAR 4 22 | NATION2_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION1_PAR 8 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .tbl 26 | DIP_READ_SPLIT_DELIMITER \| 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/10G_z1_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | #DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.HyracksPlan 4 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.HyracksPredicatePlan 5 | DIP_TOPOLOGY_NAME_PREFIX username 6 | 7 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 8 | # DIP_NUM_WORKERS 2 9 | # DIP_NUM_ACKERS 0 10 | 11 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/Z1/10G/ 12 | 13 | CUSTOMER_PAR 8 14 | ORDERS_PAR 8 15 | 16 | CUSTOMER_ORDERS_PAR 16 17 | 18 | 19 | DIP_OUTPUT_FREQ_PRINT 200000 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .tbl 23 | DIP_READ_SPLIT_DELIMITER \| 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 36 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/1G_dbtoaster_hash_hypercube_tpch5: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch5 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.HashHypercubeDBToasterTPCH5Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/1G/ 8 | 9 | REGION_PAR 1 10 | NATION_PAR 1 11 | SUPPLIER_PAR 1 12 | LINEITEM_PAR 4 13 | CUSTOMER_PAR 4 14 | ORDERS_PAR 4 15 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PAR 10 16 | FINAL_RESULT_PAR 1 17 | 18 | CUSTOMER_CARD 1500 19 | ORDERS_CARD 15000 20 | LINEITEM_CARD 60175 21 | REGION_CARD 5 22 | NATION_CARD 25 23 | SUPPLIER_CARD 100 24 | 25 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PART_SCHEME HASHHYPERCUBE 26 | 27 | DIP_OUTPUT_FREQ_PRINT 200000 28 | 29 | #below are unlikely to change 30 | DIP_EXTENSION .tbl 31 | DIP_READ_SPLIT_DELIMITER \| 32 | DIP_GLOBAL_ADD_DELIMITER | 33 | DIP_GLOBAL_SPLIT_DELIMITER \| 34 | 35 | DIP_KILL_AT_THE_END true 36 | 37 | # Storage manager parameters 38 | # Storage directory for local runs 39 | STORAGE_LOCAL_DIR /tmp/ramdisk 40 | # Storage directory for cluster runs 41 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 42 | STORAGE_COLD_START true 43 | STORAGE_MEMORY_SIZE_MB 4096 44 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/1G_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | #DIP_QUERY_PLAN ch.epfl.data.squall.query_plans.HyracksPlan 4 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.HyracksPredicatePlan 5 | DIP_TOPOLOGY_NAME_PREFIX username 6 | 7 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 8 | # DIP_NUM_WORKERS 2 9 | # DIP_NUM_ACKERS 0 10 | 11 | DIP_DATA_PATH /data/lab/squall_data/tpchdb/1G/ 12 | 13 | CUSTOMER_PAR 8 14 | ORDERS_PAR 8 15 | 16 | CUSTOMER_ORDERS_PAR 16 17 | 18 | 19 | DIP_OUTPUT_FREQ_PRINT 200000 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .tbl 23 | DIP_READ_SPLIT_DELIMITER \| 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 36 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/dbtoaster_google_failed: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME google_failed 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterGoogleMostFailedMachine 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/google_cluster_data_2011-2 8 | 9 | JOB_EVENTS_PAR 1 10 | TASK_EVENTS_PAR 8 11 | MACHINE_EVENTS_PAR 1 12 | JOB_EVENTS_TASK_EVENTS_MACHINE_EVENTS_PAR 8 13 | FINAL_RESULT_PAR 1 14 | 15 | MACHINE_EVENTS_CARD 37780 16 | JOB_EVENTS_CARD 2012240 17 | TASK_EVENTS_CARD 13829780 18 | 19 | JOB_EVENTS_TASK_EVENTS_MACHINE_EVENTS_PART_SCHEME HASHHYPERCUBE 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .csv 23 | DIP_READ_SPLIT_DELIMITER \, 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 36 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/dbtoaster_google_failed_traditional: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME google_failed 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalGoogleMostFailedMachine 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/google_cluster_data_2011-2 8 | 9 | JOB_EVENTS_PAR 1 10 | TASK_EVENTS_PAR 6 11 | MACHINE_EVENTS_PAR 1 12 | JOB_EVENTS_TASK_EVENTS_MACHINE_EVENTS_PAR 8 13 | COUNTAGG_PAR 1 14 | 15 | MACHINE_EVENTS_CARD 37780 16 | JOB_EVENTS_CARD 2012240 17 | TASK_EVENTS_CARD 13829780 18 | 19 | JOB_EVENTS_TASK_EVENTS_MACHINE_EVENTS_PART_SCHEME HASHHYPERCUBE 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .csv 23 | DIP_READ_SPLIT_DELIMITER \, 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 36 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/dbtoaster_hash_hypercube_reachability: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hash_reachability_3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.hypercube.HashHypercubeDBToasterReachability 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/link_graph 8 | 9 | ARCS1_PAR 6 10 | ARCS2_PAR 6 11 | ARCS3_PAR 6 12 | ARCS1_ARCS2_ARCS3_PAR 36 13 | 14 | COUNTAGG_PAR 36 15 | 16 | ARCS1_CARD 2043000000 17 | ARCS2_CARD 2043000000 18 | ARCS3_CARD 2043000000 19 | 20 | ARCS1_ARCS2_ARCS3_PART_SCHEME HASHHYPERCUBE 21 | 22 | DIP_INPUT_FREQ_PRINT 1000000 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .txt 26 | DIP_READ_SPLIT_DELIMITER \t 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/dbtoaster_hash_hypercube_reachability_seq: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME reachability_3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterReachabilitySeq 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/link_graph 8 | 9 | ARCS1_PAR 6 10 | ARCS2_PAR 6 11 | ARCS1_ARCS2_PAR 18 12 | ARCS3_PAR 6 13 | ARCS1_ARCS2_ARCS3_PAR 18 14 | 15 | COUNTAGG_PAR 36 16 | 17 | ARCS1_CARD 204300000 18 | ARCS2_CARD 204300000 19 | ARCS3_CARD 204300000 20 | ARCS1_ARCS2_CARD 2043000000 21 | 22 | DIP_INPUT_FREQ_PRINT 1000000 23 | 24 | ARCS1_ARCS2_PART_SCHEME HASHHYPERCUBE 25 | ARCS1_ARCS2_ARCS3_PART_SCHEME HASHHYPERCUBE 26 | 27 | #below are unlikely to change 28 | DIP_EXTENSION .txt 29 | DIP_READ_SPLIT_DELIMITER \t 30 | DIP_GLOBAL_ADD_DELIMITER | 31 | DIP_GLOBAL_SPLIT_DELIMITER \| 32 | 33 | DIP_KILL_AT_THE_END true 34 | 35 | # Storage manager parameters 36 | # Storage directory for local runs 37 | STORAGE_LOCAL_DIR /tmp/ramdisk 38 | # Storage directory for cluster runs 39 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 40 | STORAGE_COLD_START true 41 | STORAGE_MEMORY_SIZE_MB 4096 42 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/dbtoaster_hash_hypercube_url_reachability: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hash_reachability_3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.hypercube.HashHypercubeDBToasterUrlReachability 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/link_graph 8 | 9 | ARCS1_PAR 8 10 | ARCS2_PAR 8 11 | INDEX1_PAR 6 12 | ARCS1_ARCS2_INDEX1_PAR 100 13 | 14 | #COUNTAGG_PAR 36 15 | 16 | ARCS1_CARD 2043203933 17 | ARCS2_CARD 2043203933 18 | INDEX1_CARD 101717775 19 | 20 | ARCS1_ARCS2_INDEX1_PART_SCHEME HASHHYPERCUBE 21 | 22 | DIP_INPUT_FREQ_PRINT 200000 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .txt 26 | DIP_READ_SPLIT_DELIMITER \t 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/dbtoaster_hash_hypercube_url_reachability_skewed: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hash_reachability_3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.hypercube.HashHypercubeDBToasterUrlReachabilitySkewed 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/pld_graph 8 | 9 | ARCS1_PAR 25 10 | ARCS2_PAR 25 11 | INDEX1_PAR 15 12 | ARCS1_ARCS2_INDEX1_PAR 40 13 | 14 | COUNTAGG_PAR 1 15 | 16 | #ARCS1_CARD 623000000 17 | #ARCS1_CARD 10765820 18 | 19 | ARCS1_CARD 1034540 20 | 21 | #ARCS2_CARD 623000000 22 | #ARCS2_CARD 14554380 23 | 24 | ARCS2_CARD 3898580 25 | 26 | INDEX1_CARD 40000000 27 | 28 | ARCS1_ARCS2_INDEX1_PART_SCHEME HASHHYPERCUBE 29 | 30 | DIP_INPUT_FREQ_PRINT 200000 31 | 32 | #below are unlikely to change 33 | DIP_EXTENSION .txt 34 | DIP_READ_SPLIT_DELIMITER \t 35 | DIP_GLOBAL_ADD_DELIMITER | 36 | DIP_GLOBAL_SPLIT_DELIMITER \| 37 | 38 | DIP_KILL_AT_THE_END true 39 | 40 | # Storage manager parameters 41 | # Storage directory for local runs 42 | STORAGE_LOCAL_DIR /tmp/ramdisk 43 | # Storage directory for cluster runs 44 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 45 | STORAGE_COLD_START true 46 | STORAGE_MEMORY_SIZE_MB 4096 47 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/dbtoaster_hybrid_hypercube_url_reachability: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hybrid_reachability_3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.hypercube.HashHypercubeDBToasterUrlReachability 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/link_graph 8 | 9 | ARCS1_PAR 8 10 | ARCS2_PAR 8 11 | INDEX1_PAR 6 12 | ARCS1_ARCS2_INDEX1_PAR 100 13 | 14 | #COUNTAGG_PAR 36 15 | 16 | ARCS1_CARD 2043203933 17 | ARCS2_CARD 2043203933 18 | INDEX1_CARD 101717775 19 | 20 | ARCS1_ARCS2_INDEX1_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 21 | 22 | DIP_INPUT_FREQ_PRINT 200000 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .txt 26 | DIP_READ_SPLIT_DELIMITER \t 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/dbtoaster_hybrid_hypercube_url_reachability_skewed: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hash_reachability_3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.hypercube.HybridHypercubeDBToasterUrlReachabilitySkewed 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/pld_graph 8 | 9 | ARCS1_PAR 25 10 | ARCS2_PAR 25 11 | INDEX1_PAR 15 12 | ARCS1_ARCS2_INDEX1_PAR 40 13 | 14 | COUNTAGG_PAR 1 15 | 16 | #ARCS1_CARD 623000000 17 | #ARCS1_CARD 10765820 18 | 19 | ARCS1_CARD 1034540 20 | 21 | #ARCS2_CARD 623000000 22 | #ARCS2_CARD 14554380 23 | 24 | ARCS2_CARD 3898580 25 | 26 | INDEX1_CARD 40000000 27 | 28 | ARCS1_ARCS2_INDEX1_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 29 | 30 | DIP_INPUT_FREQ_PRINT 200000 31 | 32 | #below are unlikely to change 33 | DIP_EXTENSION .txt 34 | DIP_READ_SPLIT_DELIMITER \t 35 | DIP_GLOBAL_ADD_DELIMITER | 36 | DIP_GLOBAL_SPLIT_DELIMITER \| 37 | 38 | DIP_KILL_AT_THE_END true 39 | 40 | # Storage manager parameters 41 | # Storage directory for local runs 42 | STORAGE_LOCAL_DIR /tmp/ramdisk 43 | # Storage directory for cluster runs 44 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 45 | STORAGE_COLD_START true 46 | STORAGE_MEMORY_SIZE_MB 4096 47 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/dbtoaster_reachability: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME random_reachability_3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterReachability 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/link_graph 8 | 9 | ARCS1_PAR 6 10 | ARCS2_PAR 6 11 | ARCS3_PAR 6 12 | ARCS1_ARCS2_ARCS3_PAR 64 13 | 14 | COUNTAGG_PAR 1 15 | 16 | ARCS1_CARD 2043000000 17 | ARCS2_CARD 2043000000 18 | ARCS3_CARD 2043000000 19 | 20 | DIP_INPUT_FREQ_PRINT 200000 21 | 22 | #below are unlikely to change 23 | DIP_EXTENSION .txt 24 | DIP_READ_SPLIT_DELIMITER \t 25 | DIP_GLOBAL_ADD_DELIMITER | 26 | DIP_GLOBAL_SPLIT_DELIMITER \| 27 | 28 | DIP_KILL_AT_THE_END true 29 | 30 | # Storage manager parameters 31 | # Storage directory for local runs 32 | STORAGE_LOCAL_DIR /tmp/ramdisk 33 | # Storage directory for cluster runs 34 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 35 | STORAGE_COLD_START true 36 | STORAGE_MEMORY_SIZE_MB 4096 37 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/cluster/dbtoaster_reachability_seq: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME reachability_3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.dbtoaster.DBToasterReachabilitySeq 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH /data/lab/squall_data/link_graph 8 | 9 | ARCS1_PAR 6 10 | ARCS2_PAR 6 11 | ARCS1_ARCS2_PAR 12 12 | ARCS3_PAR 6 13 | ARCS1_ARCS2_ARCS3_PAR 24 14 | 15 | COUNTAGG_PAR 1 16 | 17 | ARCS1_CARD 204300000 18 | ARCS2_CARD 204300000 19 | ARCS3_CARD 204300000 20 | ARCS1_ARCS2_CARD 2043000000 21 | 22 | DIP_INPUT_FREQ_PRINT 200000 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .txt 26 | DIP_READ_SPLIT_DELIMITER \t 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/extra-local/0_01G_test_sync: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.sync.TestSync 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | DISTRIBUTION_SECS 10 11 | TUPLES_THRES 4000000 12 | ZOOKEEPER_HOST localhost:2000 13 | #WINOW_SIZE -1 14 | 15 | #ZOOKEEPER_HOST blade03:2181 16 | WINOW_SIZE 500000 17 | FREQUENCY_THRESH 100000 18 | UPDATE_RATE 100000 19 | 20 | 21 | 22 | CUSTOMER_PAR 4 23 | ORDERS_PAR 4 24 | 25 | CUSTOMER_ORDERS_PAR 1 26 | #COUNTAGG_PAR 1 27 | 28 | #below are unlikely to change 29 | DIP_EXTENSION .tbl 30 | DIP_READ_SPLIT_DELIMITER \| 31 | DIP_GLOBAL_ADD_DELIMITER | 32 | DIP_GLOBAL_SPLIT_DELIMITER \| 33 | 34 | DIP_KILL_AT_THE_END true 35 | 36 | # Storage manager parameters 37 | # Storage directory for local runs 38 | STORAGE_LOCAL_DIR /tmp/ramdisk 39 | # Storage directory for cluster runs 40 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 41 | STORAGE_COLD_START true 42 | STORAGE_MEMORY_SIZE_MB 4096 43 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/extra-local/0_01G_test_sync2: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.sync.TestSync 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | CUSTOMER_PAR 4 11 | ORDERS_PAR 4 12 | 13 | DISTRIBUTION_SECS 5 14 | TUPLES_THRES 900000 15 | ZOOKEEPER_HOST localhost:2000 16 | #WINOW_SIZE -1 17 | 18 | WINOW_SIZE 50000 19 | FREQUENCY_THRESH 5000 20 | UPDATE_RATE 1000 21 | 22 | 23 | CUSTOMER_ORDERS_PAR 1 24 | #COUNTAGG_PAR 1 25 | 26 | #below are unlikely to change 27 | DIP_EXTENSION .tbl 28 | DIP_READ_SPLIT_DELIMITER \| 29 | DIP_GLOBAL_ADD_DELIMITER | 30 | DIP_GLOBAL_SPLIT_DELIMITER \| 31 | 32 | DIP_KILL_AT_THE_END true 33 | 34 | # Storage manager parameters 35 | # Storage directory for local runs 36 | STORAGE_LOCAL_DIR /tmp/ramdisk 37 | # Storage directory for cluster runs 38 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 39 | STORAGE_COLD_START true 40 | STORAGE_MEMORY_SIZE_MB 4096 41 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/extra-local/0_01G_theta_output_dominated: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_output_dominated 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.theta.ThetaOutputDominatedPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | # THETA JOIN TYPE 11 | # STATIC PARTITIONING = 0 12 | # DYNAMIC PARTITIONING = 1 13 | DIP_JOIN_TYPE 0 14 | 15 | ORDERS_PAR 1 16 | LINEITEM_PAR 1 17 | NATION_PAR 1 18 | CUSTOMER_PAR 1 19 | SUPPLIER_PAR 1 20 | 21 | SUPPLIER_NATION_PAR 1 22 | 23 | SUPPLIER_NATION_RESHUF_PAR 1 24 | 25 | #below are unlikely to change 26 | DIP_EXTENSION .tbl 27 | DIP_READ_SPLIT_DELIMITER \| 28 | DIP_GLOBAL_ADD_DELIMITER | 29 | DIP_GLOBAL_SPLIT_DELIMITER \| 30 | 31 | DIP_KILL_AT_THE_END true 32 | 33 | 34 | NATION_CARD 50 35 | CUSTOMER_CARD 50 36 | SUPPLIER_CARD 50 37 | ORDERS_CARD 50 38 | LINEITEM_CARD 50 39 | 40 | SUPPLIER_NATION_CARD 50 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/extra-local/0_01G_tpch7HDFS_Materializer: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch7 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.TPCH7PlanHDFSMaterializer 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | NATION1_PAR 1 11 | NATION2_PAR 1 12 | CUSTOMER_PAR 1 13 | ORDERS_PAR 1 14 | SUPPLIER_PAR 1 15 | LINEITEM_PAR 1 16 | 17 | NATION2_CUSTOMER_PAR 1 18 | NATION2_CUSTOMER_ORDERS_PAR 1 19 | SUPPLIER_NATION1_PAR 1 20 | LINEITEM_SUPPLIER_NATION1_PAR 1 21 | NATION2_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION1_PAR 1 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/extra-local/0_01G_tpch7HDFS_Source: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch7 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.TPCH7Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH hdfs://localhost:9000/0.3/0.3/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | NATION1_PAR 1 11 | NATION2_PAR 1 12 | CUSTOMER_PAR 1 13 | ORDERS_PAR 4 14 | SUPPLIER_PAR 1 15 | LINEITEM_PAR 4 16 | 17 | NATION2_CUSTOMER_PAR 1 18 | NATION2_CUSTOMER_ORDERS_PAR 1 19 | SUPPLIER_NATION1_PAR 1 20 | LINEITEM_SUPPLIER_NATION1_PAR 1 21 | NATION2_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION1_PAR 1 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.HyracksPredicatePlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | CUSTOMER_PAR 1 11 | ORDERS_PAR 1 12 | 13 | CUSTOMER_ORDERS_PAR 1 14 | #COUNTAGG_PAR 1 15 | 16 | #below are unlikely to change 17 | DIP_EXTENSION .tbl 18 | DIP_READ_SPLIT_DELIMITER \| 19 | DIP_GLOBAL_ADD_DELIMITER | 20 | DIP_GLOBAL_SPLIT_DELIMITER \| 21 | 22 | DIP_KILL_AT_THE_END true 23 | 24 | # Storage manager parameters 25 | # Storage directory for local runs 26 | STORAGE_LOCAL_DIR /tmp/ramdisk 27 | # Storage directory for cluster runs 28 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 29 | STORAGE_COLD_START true 30 | STORAGE_MEMORY_SIZE_MB 4096 31 | 32 | # Storm metrics report to Graphite 33 | GRAPHITE_METRICS_CONSUMER_ENABLED false 34 | GRAPHITE_METRICS_CONSUMER_PARALELISM 1 35 | metrics.reporter.name com.verisign.storm.metrics.reporters.graphite.GraphiteReporter 36 | metrics.graphite.host 10.0.2.2 37 | metrics.graphite.port 2003 38 | metrics.graphite.prefix storm.test 39 | metrics.graphite.min-connect-attempt-interval-secs 5 40 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_hyracks_l3_batch: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.debug.HyracksL3BatchPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | CUSTOMER_PAR 1 11 | ORDERS_PAR 1 12 | 13 | CUSTOMER_ORDERS_PAR 1 14 | COUNTAGG_PAR 1 15 | 16 | #below are unlikely to change 17 | DIP_EXTENSION .tbl 18 | DIP_READ_SPLIT_DELIMITER \| 19 | DIP_GLOBAL_ADD_DELIMITER | 20 | DIP_GLOBAL_SPLIT_DELIMITER \| 21 | 22 | DIP_KILL_AT_THE_END true 23 | 24 | # Storage manager parameters 25 | # Storage directory for local runs 26 | STORAGE_LOCAL_DIR /tmp/ramdisk 27 | # Storage directory for cluster runs 28 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 29 | STORAGE_COLD_START true 30 | STORAGE_MEMORY_SIZE_MB 4096 31 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_hyracks_pre_agg: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.HyracksPreAggPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | CUSTOMER_PAR 1 11 | ORDERS_PAR 1 12 | 13 | CUSTOMER_ORDERS_PAR 1 14 | COUNTAGG_PAR 1 15 | 16 | #below are unlikely to change 17 | DIP_EXTENSION .tbl 18 | DIP_READ_SPLIT_DELIMITER \| 19 | DIP_GLOBAL_ADD_DELIMITER | 20 | DIP_GLOBAL_SPLIT_DELIMITER \| 21 | 22 | DIP_KILL_AT_THE_END true 23 | 24 | # Storage manager parameters 25 | # Storage directory for local runs 26 | STORAGE_LOCAL_DIR /tmp/ramdisk 27 | # Storage directory for cluster runs 28 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 29 | STORAGE_COLD_START true 30 | STORAGE_MEMORY_SIZE_MB 4096 31 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_scalahyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME scalaHyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.api.scala.queries.ScalaHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | CUSTOMER_PAR 1 11 | ORDERS_PAR 1 12 | 13 | CUSTOMER_ORDERS_PAR 1 14 | #COUNTAGG_PAR 1 15 | 16 | #below are unlikely to change 17 | DIP_EXTENSION .tbl 18 | DIP_READ_SPLIT_DELIMITER \| 19 | DIP_GLOBAL_ADD_DELIMITER | 20 | DIP_GLOBAL_SPLIT_DELIMITER \| 21 | 22 | DIP_KILL_AT_THE_END true 23 | 24 | # Storage manager parameters 25 | # Storage directory for local runs 26 | STORAGE_LOCAL_DIR /tmp/ramdisk 27 | # Storage directory for cluster runs 28 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 29 | STORAGE_COLD_START true 30 | STORAGE_MEMORY_SIZE_MB 4096 31 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.theta.ThetaHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | # THETA JOIN TYPE 11 | # STATIC PARTITIONING = 0 12 | # DYNAMIC PARTITIONING = 1 13 | DIP_JOIN_TYPE 0 14 | 15 | CUSTOMER_PAR 2 16 | ORDERS_PAR 2 17 | 18 | CUSTOMER_ORDERS_PAR 4 19 | COUNTAGG_PAR 1 20 | 21 | CUSTOMER_ORDERS_RESHUF_PAR 2 22 | 23 | CUSTOMER_CARD 50 24 | ORDERS_CARD 50 25 | 26 | #below are unlikely to change 27 | DIP_EXTENSION .tbl 28 | DIP_READ_SPLIT_DELIMITER \| 29 | DIP_GLOBAL_ADD_DELIMITER | 30 | DIP_GLOBAL_SPLIT_DELIMITER \| 31 | 32 | DIP_KILL_AT_THE_END true 33 | 34 | # Storage manager parameters 35 | # Storage directory for local runs 36 | STORAGE_LOCAL_DIR /tmp/ramdisk 37 | # Storage directory for cluster runs 38 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 39 | STORAGE_COLD_START true 40 | STORAGE_MEMORY_SIZE_MB 4096 41 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_theta_input_dominated: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_input_dominated 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.theta.ThetaInputDominatedPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | # THETA JOIN TYPE 11 | # STATIC PARTITIONING = 0 12 | # DYNAMIC PARTITIONING = 1 13 | DIP_JOIN_TYPE 1 14 | 15 | ORDERS_PAR 2 16 | LINEITEM_PAR 2 17 | 18 | LINEITEM_ORDERS_PAR 4 19 | 20 | LINEITEM_ORDERS_RESHUF_PAR 2 21 | 22 | #below are unlikely to change 23 | DIP_EXTENSION .tbl 24 | DIP_READ_SPLIT_DELIMITER \| 25 | DIP_GLOBAL_ADD_DELIMITER | 26 | DIP_GLOBAL_SPLIT_DELIMITER \| 27 | 28 | DIP_KILL_AT_THE_END true 29 | 30 | 31 | NATION_CARD 50 32 | CUSTOMER_CARD 50 33 | SUPPLIER_CARD 50 34 | ORDERS_CARD 50 35 | LINEITEM_CARD 50 36 | 37 | LINEITEM_ORDERS_CARD 50 38 | 39 | # Storage manager parameters 40 | # Storage directory for local runs 41 | STORAGE_LOCAL_DIR /tmp/ramdisk 42 | # Storage directory for cluster runs 43 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 44 | STORAGE_COLD_START true 45 | STORAGE_MEMORY_SIZE_MB 4096 46 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_theta_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.theta.ThetaTPCH3Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | DIP_JOIN_TYPE 0 11 | ADVISOR_TYPE 1 12 | EPSILON 0.1 13 | 14 | CUSTOMER_PAR 1 15 | ORDERS_PAR 1 16 | LINEITEM_PAR 1 17 | 18 | CUSTOMER_ORDERS_PAR 2 19 | CUSTOMER_ORDERS_LINEITEM_PAR 1 20 | 21 | CUSTOMER_CARD 15 22 | ORDERS_CARD 150 23 | LINEITEM_CARD 600 24 | CUSTOMER_ORDERS_CARD 150 25 | 26 | #below are unlikely to change 27 | DIP_EXTENSION .tbl 28 | DIP_READ_SPLIT_DELIMITER \| 29 | DIP_GLOBAL_ADD_DELIMITER | 30 | DIP_GLOBAL_SPLIT_DELIMITER \| 31 | 32 | DIP_KILL_AT_THE_END true 33 | 34 | # Storage manager parameters 35 | # Storage directory for local runs 36 | STORAGE_LOCAL_DIR /tmp/ramdisk 37 | # Storage directory for cluster runs 38 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 39 | STORAGE_COLD_START true 40 | STORAGE_MEMORY_SIZE_MB 4096 41 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_theta_tpch4: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch4 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.theta.ThetaTPCH4Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | #JOIN TYPE 0 for Static, 1 for dynamic 11 | DIP_JOIN_TYPE 1 12 | 13 | #0 for heuristics and 1 for theoretical 14 | ADVISOR_TYPE 1 15 | EPSILON 0.1 16 | 17 | 18 | ORDERS_PAR 2 19 | LINEITEM_PAR 1 20 | 21 | ORDERS_LINEITEM_PAR 4 22 | 23 | FINAL_RESULT_PAR 4 24 | 25 | # Define Static Theta Joins parameters Here: 26 | ORDERS_CARD 50 27 | LINEITEM_CARD 50 28 | 29 | ORDERS_LINEITEM_CARD 50 30 | 31 | 32 | 33 | # Define Dynamic Theta Joins parameters Here: 34 | ORDERS_LINEITEM_RESHUF_PAR 2 35 | DIP_THETA_CLOCK_REFRESH_RATE_MILLISECONDS 100 36 | 37 | 38 | #below are unlikely to change 39 | DIP_EXTENSION .tbl 40 | DIP_READ_SPLIT_DELIMITER \| 41 | DIP_GLOBAL_ADD_DELIMITER | 42 | DIP_GLOBAL_SPLIT_DELIMITER \| 43 | 44 | DIP_KILL_AT_THE_END true 45 | 46 | # Storage manager parameters 47 | # Storage directory for local runs 48 | STORAGE_LOCAL_DIR /tmp/ramdisk 49 | # Storage directory for cluster runs 50 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 51 | STORAGE_COLD_START true 52 | STORAGE_MEMORY_SIZE_MB 4096 53 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_tpch10: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch10 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.TPCH10Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | CUSTOMER_PAR 1 11 | ORDERS_PAR 1 12 | NATION_PAR 1 13 | LINEITEM_PAR 1 14 | 15 | CUSTOMER_ORDERS_PAR 1 16 | CUSTOMER_ORDERS_NATION_PAR 1 17 | CUSTOMER_ORDERS_NATION_LINEITEM_PAR 1 18 | 19 | #below are unlikely to change 20 | DIP_EXTENSION .tbl 21 | DIP_READ_SPLIT_DELIMITER \| 22 | DIP_GLOBAL_ADD_DELIMITER | 23 | DIP_GLOBAL_SPLIT_DELIMITER \| 24 | 25 | DIP_KILL_AT_THE_END true 26 | 27 | # Storage manager parameters 28 | # Storage directory for local runs 29 | STORAGE_LOCAL_DIR /tmp/ramdisk 30 | # Storage directory for cluster runs 31 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 32 | STORAGE_COLD_START true 33 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.TPCH3Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | CUSTOMER_PAR 1 11 | ORDERS_PAR 1 12 | LINEITEM_PAR 1 13 | 14 | CUSTOMER_ORDERS_PAR 1 15 | CUSTOMER_ORDERS_LINEITEM_PAR 1 16 | 17 | #below are unlikely to change 18 | DIP_EXTENSION .tbl 19 | DIP_READ_SPLIT_DELIMITER \| 20 | DIP_GLOBAL_ADD_DELIMITER | 21 | DIP_GLOBAL_SPLIT_DELIMITER \| 22 | 23 | DIP_KILL_AT_THE_END true 24 | 25 | # Storage manager parameters 26 | # Storage directory for local runs 27 | STORAGE_LOCAL_DIR /tmp/ramdisk 28 | # Storage directory for cluster runs 29 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 30 | STORAGE_COLD_START true 31 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_tpch4: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch4 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.TPCH4Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | ORDERS_PAR 1 11 | LINEITEM_PAR 1 12 | 13 | ORDERS_LINEITEM_PAR 1 14 | FINAL_RESULT_PAR 1 15 | 16 | #below are unlikely to change 17 | DIP_EXTENSION .tbl 18 | DIP_READ_SPLIT_DELIMITER \| 19 | DIP_GLOBAL_ADD_DELIMITER | 20 | DIP_GLOBAL_SPLIT_DELIMITER \| 21 | 22 | DIP_KILL_AT_THE_END true 23 | 24 | # Storage manager parameters 25 | # Storage directory for local runs 26 | STORAGE_LOCAL_DIR /tmp/ramdisk 27 | # Storage directory for cluster runs 28 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 29 | STORAGE_COLD_START true 30 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_tpch5: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch5 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.TPCH5Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | REGION_PAR 1 11 | NATION_PAR 1 12 | SUPPLIER_PAR 1 13 | LINEITEM_PAR 1 14 | CUSTOMER_PAR 1 15 | ORDERS_PAR 1 16 | 17 | REGION_NATION_PAR 1 18 | REGION_NATION_SUPPLIER_PAR 1 19 | REGION_NATION_SUPPLIER_LINEITEM_PAR 1 20 | CUSTOMER_ORDERS_PAR 1 21 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PAR 1 22 | FINAL_RESULT_PAR 1 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .tbl 26 | DIP_READ_SPLIT_DELIMITER \| 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_tpch5avg: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch5avg 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.debug.TPCH5PlanAvg 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | REGION_PAR 1 11 | NATION_PAR 1 12 | SUPPLIER_PAR 1 13 | LINEITEM_PAR 1 14 | CUSTOMER_PAR 1 15 | ORDERS_PAR 1 16 | 17 | REGION_NATION_PAR 1 18 | REGION_NATION_SUPPLIER_PAR 1 19 | REGION_NATION_SUPPLIER_LINEITEM_PAR 1 20 | CUSTOMER_ORDERS_PAR 1 21 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PAR 1 22 | FINAL_RESULT_PAR 1 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .tbl 26 | DIP_READ_SPLIT_DELIMITER \| 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_tpch7: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch7 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.TPCH7Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | NATION1_PAR 1 11 | NATION2_PAR 1 12 | CUSTOMER_PAR 1 13 | ORDERS_PAR 1 14 | SUPPLIER_PAR 1 15 | LINEITEM_PAR 1 16 | 17 | NATION2_CUSTOMER_PAR 1 18 | NATION2_CUSTOMER_ORDERS_PAR 1 19 | SUPPLIER_NATION1_PAR 1 20 | LINEITEM_SUPPLIER_NATION1_PAR 1 21 | NATION2_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION1_PAR 1 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_tpch8: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch8 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.TPCH8Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | REGION_PAR 1 11 | NATION1_PAR 1 12 | NATION2_PAR 1 13 | CUSTOMER_PAR 1 14 | SUPPLIER_PAR 1 15 | PART_PAR 1 16 | LINEITEM_PAR 1 17 | ORDERS_PAR 1 18 | 19 | REGION_NATION1_PAR 1 20 | REGION_NATION1_CUSTOMER_PAR 1 21 | 22 | SUPPLIER_NATION2_PAR 1 23 | 24 | PART_LINEITEM_PAR 1 25 | PART_LINEITEM_ORDERS_PAR 1 26 | SUPPLIER_NATION2_PART_LINEITEM_ORDERS_PAR 1 27 | 28 | REGION_NATION1_CUSTOMER_SUPPLIER_NATION2_PART_LINEITEM_ORDERS_PAR 1 29 | 30 | #below are unlikely to change 31 | DIP_EXTENSION .tbl 32 | DIP_READ_SPLIT_DELIMITER \| 33 | DIP_GLOBAL_ADD_DELIMITER | 34 | DIP_GLOBAL_SPLIT_DELIMITER \| 35 | 36 | DIP_KILL_AT_THE_END true 37 | 38 | # Storage manager parameters 39 | # Storage directory for local runs 40 | STORAGE_LOCAL_DIR /tmp/ramdisk 41 | # Storage directory for cluster runs 42 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 43 | STORAGE_COLD_START true 44 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_tpch9: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch9 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.TPCH9Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | PART_PAR 1 11 | LINEITEM_PAR 1 12 | PARTSUPP_PAR 1 13 | ORDERS_PAR 1 14 | SUPPLIER_PAR 1 15 | NATION_PAR 1 16 | 17 | PART_LINEITEM_PAR 1 18 | PART_LINEITEM_PARTSUPP_PAR 1 19 | PART_LINEITEM_PARTSUPP_ORDERS_PAR 1 20 | PART_LINEITEM_PARTSUPP_ORDERS_SUPPLIER_PAR 1 21 | PART_LINEITEM_PARTSUPP_ORDERS_SUPPLIER_NATION_PAR 1 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_traditional_hash_hypercube_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TradionalHypercubeThetaHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | DIP_JOIN_TYPE 0 11 | 12 | CUSTOMER_PAR 2 13 | ORDERS_PAR 2 14 | 15 | CUSTOMER_ORDERS_PAR 8 16 | COUNTAGG_PAR 1 17 | 18 | CUSTOMER_CARD 50 19 | ORDERS_CARD 50 20 | 21 | HYPERCUBE_ENABLED true 22 | CUSTOMER_ORDERS_PART_SCHEME HASHHYPERCUBE 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .tbl 26 | DIP_READ_SPLIT_DELIMITER \| 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_traditional_hash_hypercube_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalHashTPCH3Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | CUSTOMER_PAR 1 11 | ORDERS_PAR 1 12 | LINEITEM_PAR 1 13 | CUSTOMER_ORDERS_LINEITEM_PAR 1 14 | COUNTAGG_PAR 1 15 | 16 | CUSTOMER_CARD 1500 17 | ORDERS_CARD 15000 18 | LINEITEM_CARD 60175 19 | 20 | CUSTOMER_ORDERS_LINEITEM_PART_SCHEME HASHHYPERCUBE 21 | 22 | #below are unlikely to change 23 | DIP_EXTENSION .tbl 24 | DIP_READ_SPLIT_DELIMITER \| 25 | DIP_GLOBAL_ADD_DELIMITER | 26 | DIP_GLOBAL_SPLIT_DELIMITER \| 27 | 28 | DIP_KILL_AT_THE_END true 29 | 30 | # Storage manager parameters 31 | # Storage directory for local runs 32 | STORAGE_LOCAL_DIR /tmp/ramdisk 33 | # Storage directory for cluster runs 34 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 35 | STORAGE_COLD_START true 36 | STORAGE_MEMORY_SIZE_MB 4096 37 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_traditional_hybrid_hypercube_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TradionalHypercubeThetaHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | DIP_JOIN_TYPE 0 11 | 12 | CUSTOMER_PAR 2 13 | ORDERS_PAR 2 14 | 15 | CUSTOMER_ORDERS_PAR 8 16 | COUNTAGG_PAR 1 17 | 18 | CUSTOMER_CARD 50 19 | ORDERS_CARD 50 20 | 21 | HYPERCUBE_ENABLED true 22 | CUSTOMER_ORDERS_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .tbl 26 | DIP_READ_SPLIT_DELIMITER \| 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_traditional_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | 11 | CUSTOMER_PAR 2 12 | ORDERS_PAR 2 13 | 14 | CUSTOMER_ORDERS_PAR 8 15 | COUNTAGG_PAR 1 16 | 17 | CUSTOMER_CARD 50 18 | ORDERS_CARD 50 19 | 20 | #below are unlikely to change 21 | DIP_EXTENSION .tbl 22 | DIP_READ_SPLIT_DELIMITER \| 23 | DIP_GLOBAL_ADD_DELIMITER | 24 | DIP_GLOBAL_SPLIT_DELIMITER \| 25 | 26 | DIP_KILL_AT_THE_END true 27 | 28 | # Storage manager parameters 29 | # Storage directory for local runs 30 | STORAGE_LOCAL_DIR /tmp/ramdisk 31 | # Storage directory for cluster runs 32 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 33 | STORAGE_COLD_START true 34 | STORAGE_MEMORY_SIZE_MB 4096 35 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_traditional_random_hypercube_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TradionalHypercubeThetaHyracksPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | DIP_JOIN_TYPE 0 11 | 12 | CUSTOMER_PAR 2 13 | ORDERS_PAR 2 14 | 15 | CUSTOMER_ORDERS_PAR 8 16 | COUNTAGG_PAR 1 17 | 18 | CUSTOMER_CARD 50 19 | ORDERS_CARD 50 20 | 21 | HYPERCUBE_ENABLED true 22 | CUSTOMER_ORDERS_PART_SCHEME HYPERCUBE 23 | 24 | #below are unlikely to change 25 | DIP_EXTENSION .tbl 26 | DIP_READ_SPLIT_DELIMITER \| 27 | DIP_GLOBAL_ADD_DELIMITER | 28 | DIP_GLOBAL_SPLIT_DELIMITER \| 29 | 30 | DIP_KILL_AT_THE_END true 31 | 32 | # Storage manager parameters 33 | # Storage directory for local runs 34 | STORAGE_LOCAL_DIR /tmp/ramdisk 35 | # Storage directory for cluster runs 36 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 37 | STORAGE_COLD_START true 38 | STORAGE_MEMORY_SIZE_MB 4096 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_traditional_tpch3: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalTPCH3Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | CUSTOMER_PAR 1 11 | ORDERS_PAR 1 12 | LINEITEM_PAR 1 13 | CUSTOMER_ORDERS_LINEITEM_PAR 1 14 | COUNTAGG_PAR 1 15 | 16 | CUSTOMER_CARD 1500 17 | ORDERS_CARD 15000 18 | LINEITEM_CARD 60175 19 | 20 | #below are unlikely to change 21 | DIP_EXTENSION .tbl 22 | DIP_READ_SPLIT_DELIMITER \| 23 | DIP_GLOBAL_ADD_DELIMITER | 24 | DIP_GLOBAL_SPLIT_DELIMITER \| 25 | 26 | DIP_KILL_AT_THE_END true 27 | 28 | # Storage manager parameters 29 | # Storage directory for local runs 30 | STORAGE_LOCAL_DIR /tmp/ramdisk 31 | # Storage directory for cluster runs 32 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 33 | STORAGE_COLD_START true 34 | STORAGE_MEMORY_SIZE_MB 4096 35 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/0_01G_traditional_tpch5: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch5 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalTPCH5Plan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | REGION_PAR 1 11 | NATION_PAR 1 12 | SUPPLIER_PAR 1 13 | LINEITEM_PAR 1 14 | CUSTOMER_PAR 1 15 | ORDERS_PAR 1 16 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PAR 2 17 | FINAL_RESULT_PAR 1 18 | 19 | CUSTOMER_CARD 1500 20 | ORDERS_CARD 15000 21 | LINEITEM_CARD 60175 22 | REGION_CARD 5 23 | NATION_CARD 25 24 | SUPPLIER_CARD 100 25 | 26 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PART_SCHEME HYPERCUBE 27 | 28 | #below are unlikely to change 29 | DIP_EXTENSION .tbl 30 | DIP_READ_SPLIT_DELIMITER \| 31 | DIP_GLOBAL_ADD_DELIMITER | 32 | DIP_GLOBAL_SPLIT_DELIMITER \| 33 | 34 | DIP_KILL_AT_THE_END true 35 | 36 | # Storage manager parameters 37 | # Storage directory for local runs 38 | STORAGE_LOCAL_DIR /tmp/ramdisk 39 | # Storage directory for cluster runs 40 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 41 | STORAGE_COLD_START true 42 | STORAGE_MEMORY_SIZE_MB 4096 43 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/10K_rst: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME rst 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.shj.RSTPlan 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/rst/10K/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | R_PAR 1 11 | S_PAR 1 12 | T_PAR 1 13 | 14 | R_S_PAR 2 15 | R_S_T_PAR 4 16 | 17 | # below are unlikely to change 18 | DIP_EXTENSION .dat 19 | DIP_READ_SPLIT_DELIMITER , 20 | DIP_GLOBAL_ADD_DELIMITER | 21 | DIP_GLOBAL_SPLIT_DELIMITER \| 22 | 23 | DIP_KILL_AT_THE_END true 24 | 25 | # Storage manager parameters 26 | # Storage directory for local runs 27 | STORAGE_LOCAL_DIR /tmp/ramdisk 28 | # Storage directory for cluster runs 29 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 30 | STORAGE_COLD_START true 31 | STORAGE_MEMORY_SIZE_MB 4096 32 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/traditional_reachability: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME reachability_3 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalHybridReachability 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/link_graph/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | ARCS1_PAR 1 11 | ARCS2_PAR 1 12 | ARCS3_PAR 1 13 | ARCS1_ARCS2_ARCS3_PAR 8 14 | 15 | COUNTAGG_PAR 1 16 | 17 | ARCS1_CARD 2043000000 18 | ARCS2_CARD 2043000000 19 | ARCS3_CARD 2043000000 20 | 21 | ARCS1_ARCS2_ARCS3_PART_SCHEME BRUTEFORCEHYBRIDHYPERCUBE 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .txt 25 | DIP_READ_SPLIT_DELIMITER \t 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 36 | STORAGE_COLD_START true 37 | STORAGE_MEMORY_SIZE_MB 4096 38 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/local/traditional_url_reachability: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME url_reachability 3 | DIP_QUERY_PLAN ch.epfl.data.squall.examples.imperative.traditional.TraditionalHybridUrlReachability 4 | DIP_TOPOLOGY_NAME_PREFIX username 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../test/data/link_graph/0.01G/ 8 | DIP_RESULT_ROOT ../test/results/ 9 | 10 | ARCS1_PAR 1 11 | ARCS2_PAR 1 12 | INDEX1_PAR 1 13 | ARCS1_ARCS2_INDEX1_PAR 8 14 | 15 | COUNTAGG_PAR 1 16 | 17 | ARCS1_CARD 2043000000 18 | ARCS2_CARD 2043000000 19 | INDEX1_CARD 204300000 20 | 21 | #below are unlikely to change 22 | DIP_EXTENSION .txt 23 | DIP_READ_SPLIT_DELIMITER \t 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | 29 | # Storage manager parameters 30 | # Storage directory for local runs 31 | STORAGE_LOCAL_DIR /tmp/ramdisk 32 | # Storage directory for cluster runs 33 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 34 | STORAGE_COLD_START true 35 | STORAGE_MEMORY_SIZE_MB 4096 36 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/push/local/console-run/0_01G_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_NUM_ACKERS 0 5 | 6 | DIP_DATA_PATH ../test/data/tpch/0.01G/ 7 | DIP_RESULT_ROOT ../test/results/ 8 | 9 | DIP_STATISTIC_COLLECTOR false 10 | DIP_KEY_REGION_ROOT ../test/m_bucket/key_region 11 | 12 | # THETA-JOIN TYPE 13 | # Content Insensitive 14 | # STATIC_CIS = 0; 15 | # EPOCHS_CIS = 1; 16 | # Content sensitive counterparts 17 | # STATIC_CS = 2; 18 | # EPOCHS_CS = 3; 19 | DIP_JOIN_TYPE 2 20 | 21 | # Parallelisms 22 | CUSTOMER_PAR 1 23 | ORDERS_PAR 1 24 | 25 | CUSTOMER_ORDERS_PAR 8 26 | 27 | # Define Dynamic 1-Bucket Theta Joins parameters Here: 28 | CUSTOMER_ORDERS_RESHUF_PAR 8 29 | 30 | # Define Static 1-Bucket Theta Joins parameters Here: 31 | CUSTOMER_CARD 50 32 | ORDERS_CARD 50 33 | 34 | #below are unlikely to change 35 | DIP_EXTENSION .tbl 36 | DIP_READ_SPLIT_DELIMITER \| 37 | DIP_GLOBAL_ADD_DELIMITER | 38 | DIP_GLOBAL_SPLIT_DELIMITER \| 39 | 40 | DIP_KILL_AT_THE_END true 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/push/local/console-run/0_01G_z1_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_NUM_ACKERS 0 5 | 6 | DIP_DATA_PATH ../test/data/tpch/0.01G_z1/ 7 | DIP_RESULT_ROOT ../test/results/ 8 | 9 | DIP_STATISTIC_COLLECTOR false 10 | DIP_KEY_REGION_ROOT ../test/m_bucket/key_region 11 | 12 | # THETA-JOIN TYPE 13 | # Content Insensitive 14 | # STATIC_CIS = 0; 15 | # EPOCHS_CIS = 1; 16 | # Content sensitive counterparts 17 | # STATIC_CS = 2; 18 | # EPOCHS_CS = 3; 19 | DIP_JOIN_TYPE 2 20 | 21 | # Parallelisms 22 | CUSTOMER_PAR 1 23 | ORDERS_PAR 1 24 | 25 | CUSTOMER_ORDERS_PAR 8 26 | 27 | # Define Dynamic 1-Bucket Theta Joins parameters Here: 28 | CUSTOMER_ORDERS_RESHUF_PAR 8 29 | 30 | # Define Static 1-Bucket Theta Joins parameters Here: 31 | CUSTOMER_CARD 50 32 | ORDERS_CARD 50 33 | 34 | #below are unlikely to change 35 | DIP_EXTENSION .tbl 36 | DIP_READ_SPLIT_DELIMITER \| 37 | DIP_GLOBAL_ADD_DELIMITER | 38 | DIP_GLOBAL_SPLIT_DELIMITER \| 39 | 40 | DIP_KILL_AT_THE_END true 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/push/local/console-run/0_01G_z2_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_NUM_ACKERS 0 5 | 6 | DIP_DATA_PATH ../test/data/tpch/0.01G_z2/ 7 | DIP_RESULT_ROOT ../test/results/ 8 | 9 | DIP_STATISTIC_COLLECTOR false 10 | DIP_KEY_REGION_ROOT ../test/m_bucket/key_region 11 | 12 | # THETA-JOIN TYPE 13 | # Content Insensitive 14 | # STATIC_CIS = 0; 15 | # EPOCHS_CIS = 1; 16 | # Content sensitive counterparts 17 | # STATIC_CS = 2; 18 | # EPOCHS_CS = 3; 19 | DIP_JOIN_TYPE 2 20 | 21 | # Parallelisms 22 | CUSTOMER_PAR 1 23 | ORDERS_PAR 1 24 | 25 | CUSTOMER_ORDERS_PAR 8 26 | 27 | # Define Dynamic 1-Bucket Theta Joins parameters Here: 28 | CUSTOMER_ORDERS_RESHUF_PAR 8 29 | 30 | # Define Static 1-Bucket Theta Joins parameters Here: 31 | CUSTOMER_CARD 50 32 | ORDERS_CARD 50 33 | 34 | #below are unlikely to change 35 | DIP_EXTENSION .tbl 36 | DIP_READ_SPLIT_DELIMITER \| 37 | DIP_GLOBAL_ADD_DELIMITER | 38 | DIP_GLOBAL_SPLIT_DELIMITER \| 39 | 40 | DIP_KILL_AT_THE_END true 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/push/local/console-run/0_01G_z3_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_NUM_ACKERS 0 5 | 6 | DIP_DATA_PATH ../test/data/tpch/0.01G_z3/ 7 | DIP_RESULT_ROOT ../test/results/ 8 | 9 | DIP_STATISTIC_COLLECTOR false 10 | DIP_KEY_REGION_ROOT ../test/m_bucket/key_region 11 | 12 | # THETA-JOIN TYPE 13 | # Content Insensitive 14 | # STATIC_CIS = 0; 15 | # EPOCHS_CIS = 1; 16 | # Content sensitive counterparts 17 | # STATIC_CS = 2; 18 | # EPOCHS_CS = 3; 19 | DIP_JOIN_TYPE 2 20 | 21 | # Parallelisms 22 | CUSTOMER_PAR 1 23 | ORDERS_PAR 1 24 | 25 | CUSTOMER_ORDERS_PAR 8 26 | 27 | # Define Dynamic 1-Bucket Theta Joins parameters Here: 28 | CUSTOMER_ORDERS_RESHUF_PAR 8 29 | 30 | # Define Static 1-Bucket Theta Joins parameters Here: 31 | CUSTOMER_CARD 50 32 | ORDERS_CARD 50 33 | 34 | #below are unlikely to change 35 | DIP_EXTENSION .tbl 36 | DIP_READ_SPLIT_DELIMITER \| 37 | DIP_GLOBAL_ADD_DELIMITER | 38 | DIP_GLOBAL_SPLIT_DELIMITER \| 39 | 40 | DIP_KILL_AT_THE_END true 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/push/local/console-run/0_01G_z4_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_NUM_ACKERS 0 5 | 6 | DIP_DATA_PATH ../test/data/tpch/0.01G_z4/ 7 | DIP_RESULT_ROOT ../test/results/ 8 | 9 | DIP_STATISTIC_COLLECTOR false 10 | DIP_KEY_REGION_ROOT ../test/m_bucket/key_region 11 | 12 | # THETA-JOIN TYPE 13 | # Content Insensitive 14 | # STATIC_CIS = 0; 15 | # EPOCHS_CIS = 1; 16 | # Content sensitive counterparts 17 | # STATIC_CS = 2; 18 | # EPOCHS_CS = 3; 19 | DIP_JOIN_TYPE 2 20 | 21 | # Parallelisms 22 | CUSTOMER_PAR 1 23 | ORDERS_PAR 1 24 | 25 | CUSTOMER_ORDERS_PAR 8 26 | 27 | # Define Dynamic 1-Bucket Theta Joins parameters Here: 28 | CUSTOMER_ORDERS_RESHUF_PAR 8 29 | 30 | # Define Static 1-Bucket Theta Joins parameters Here: 31 | CUSTOMER_CARD 50 32 | ORDERS_CARD 50 33 | 34 | #below are unlikely to change 35 | DIP_EXTENSION .tbl 36 | DIP_READ_SPLIT_DELIMITER \| 37 | DIP_GLOBAL_ADD_DELIMITER | 38 | DIP_GLOBAL_SPLIT_DELIMITER \| 39 | 40 | DIP_KILL_AT_THE_END true 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/push/local/eclipse-run/0_01G_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_NUM_ACKERS 0 5 | 6 | DIP_DATA_PATH test/data/tpch/0.01G/ 7 | DIP_RESULT_ROOT test/results/ 8 | 9 | DIP_STATISTIC_COLLECTOR false 10 | DIP_KEY_REGION_ROOT test/m_bucket/key_region 11 | 12 | # THETA-JOIN TYPE 13 | # Content Insensitive 14 | # STATIC_CIS = 0; 15 | # EPOCHS_CIS = 1; 16 | # Content sensitive counterparts 17 | # STATIC_CS = 2; 18 | # EPOCHS_CS = 3; 19 | DIP_JOIN_TYPE 2 20 | 21 | # Parallelisms 22 | CUSTOMER_PAR 1 23 | ORDERS_PAR 1 24 | 25 | CUSTOMER_ORDERS_PAR 8 26 | 27 | # Define Dynamic 1-Bucket Theta Joins parameters Here: 28 | CUSTOMER_ORDERS_RESHUF_PAR 8 29 | 30 | # Define Static 1-Bucket Theta Joins parameters Here: 31 | CUSTOMER_CARD 50 32 | ORDERS_CARD 50 33 | 34 | #below are unlikely to change 35 | DIP_EXTENSION .tbl 36 | DIP_READ_SPLIT_DELIMITER \| 37 | DIP_GLOBAL_ADD_DELIMITER | 38 | DIP_GLOBAL_SPLIT_DELIMITER \| 39 | 40 | DIP_KILL_AT_THE_END true 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/push/local/eclipse-run/0_01G_z1_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_NUM_ACKERS 0 5 | 6 | DIP_DATA_PATH test/data/tpch/0.01G_z1/ 7 | DIP_RESULT_ROOT test/results/ 8 | 9 | DIP_STATISTIC_COLLECTOR false 10 | DIP_KEY_REGION_ROOT test/m_bucket/key_region 11 | 12 | # THETA-JOIN TYPE 13 | # Content Insensitive 14 | # STATIC_CIS = 0; 15 | # EPOCHS_CIS = 1; 16 | # Content sensitive counterparts 17 | # STATIC_CS = 2; 18 | # EPOCHS_CS = 3; 19 | DIP_JOIN_TYPE 2 20 | 21 | # Parallelisms 22 | CUSTOMER_PAR 1 23 | ORDERS_PAR 1 24 | 25 | CUSTOMER_ORDERS_PAR 8 26 | 27 | # Define Dynamic 1-Bucket Theta Joins parameters Here: 28 | CUSTOMER_ORDERS_RESHUF_PAR 8 29 | 30 | # Define Static 1-Bucket Theta Joins parameters Here: 31 | CUSTOMER_CARD 50 32 | ORDERS_CARD 50 33 | 34 | #below are unlikely to change 35 | DIP_EXTENSION .tbl 36 | DIP_READ_SPLIT_DELIMITER \| 37 | DIP_GLOBAL_ADD_DELIMITER | 38 | DIP_GLOBAL_SPLIT_DELIMITER \| 39 | 40 | DIP_KILL_AT_THE_END true 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/push/local/eclipse-run/0_01G_z2_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_NUM_ACKERS 0 5 | 6 | DIP_DATA_PATH test/data/tpch/0.01G_z2/ 7 | DIP_RESULT_ROOT test/results/ 8 | 9 | DIP_STATISTIC_COLLECTOR false 10 | DIP_KEY_REGION_ROOT test/m_bucket/key_region 11 | 12 | # THETA-JOIN TYPE 13 | # Content Insensitive 14 | # STATIC_CIS = 0; 15 | # EPOCHS_CIS = 1; 16 | # Content sensitive counterparts 17 | # STATIC_CS = 2; 18 | # EPOCHS_CS = 3; 19 | DIP_JOIN_TYPE 2 20 | 21 | # Parallelisms 22 | CUSTOMER_PAR 1 23 | ORDERS_PAR 1 24 | 25 | CUSTOMER_ORDERS_PAR 8 26 | 27 | # Define Dynamic 1-Bucket Theta Joins parameters Here: 28 | CUSTOMER_ORDERS_RESHUF_PAR 8 29 | 30 | # Define Static 1-Bucket Theta Joins parameters Here: 31 | CUSTOMER_CARD 50 32 | ORDERS_CARD 50 33 | 34 | #below are unlikely to change 35 | DIP_EXTENSION .tbl 36 | DIP_READ_SPLIT_DELIMITER \| 37 | DIP_GLOBAL_ADD_DELIMITER | 38 | DIP_GLOBAL_SPLIT_DELIMITER \| 39 | 40 | DIP_KILL_AT_THE_END true 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/push/local/eclipse-run/0_01G_z3_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_NUM_ACKERS 0 5 | 6 | DIP_DATA_PATH test/data/tpch/0.01G_z3/ 7 | DIP_RESULT_ROOT test/results/ 8 | 9 | DIP_STATISTIC_COLLECTOR false 10 | DIP_KEY_REGION_ROOT test/m_bucket/key_region 11 | 12 | # THETA-JOIN TYPE 13 | # Content Insensitive 14 | # STATIC_CIS = 0; 15 | # EPOCHS_CIS = 1; 16 | # Content sensitive counterparts 17 | # STATIC_CS = 2; 18 | # EPOCHS_CS = 3; 19 | DIP_JOIN_TYPE 2 20 | 21 | # Parallelisms 22 | CUSTOMER_PAR 1 23 | ORDERS_PAR 1 24 | 25 | CUSTOMER_ORDERS_PAR 8 26 | 27 | # Define Dynamic 1-Bucket Theta Joins parameters Here: 28 | CUSTOMER_ORDERS_RESHUF_PAR 8 29 | 30 | # Define Static 1-Bucket Theta Joins parameters Here: 31 | CUSTOMER_CARD 50 32 | ORDERS_CARD 50 33 | 34 | #below are unlikely to change 35 | DIP_EXTENSION .tbl 36 | DIP_READ_SPLIT_DELIMITER \| 37 | DIP_GLOBAL_ADD_DELIMITER | 38 | DIP_GLOBAL_SPLIT_DELIMITER \| 39 | 40 | DIP_KILL_AT_THE_END true 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/push/local/eclipse-run/0_01G_z4_theta_hyracks: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_NUM_ACKERS 0 5 | 6 | DIP_DATA_PATH test/data/tpch/0.01G_z4/ 7 | DIP_RESULT_ROOT test/results/ 8 | 9 | DIP_STATISTIC_COLLECTOR false 10 | DIP_KEY_REGION_ROOT test/m_bucket/key_region 11 | 12 | # THETA-JOIN TYPE 13 | # Content Insensitive 14 | # STATIC_CIS = 0; 15 | # EPOCHS_CIS = 1; 16 | # Content sensitive counterparts 17 | # STATIC_CS = 2; 18 | # EPOCHS_CS = 3; 19 | DIP_JOIN_TYPE 2 20 | 21 | # Parallelisms 22 | CUSTOMER_PAR 1 23 | ORDERS_PAR 1 24 | 25 | CUSTOMER_ORDERS_PAR 8 26 | 27 | # Define Dynamic 1-Bucket Theta Joins parameters Here: 28 | CUSTOMER_ORDERS_RESHUF_PAR 8 29 | 30 | # Define Static 1-Bucket Theta Joins parameters Here: 31 | CUSTOMER_CARD 50 32 | ORDERS_CARD 50 33 | 34 | #below are unlikely to change 35 | DIP_EXTENSION .tbl 36 | DIP_READ_SPLIT_DELIMITER \| 37 | DIP_GLOBAL_ADD_DELIMITER | 38 | DIP_GLOBAL_SPLIT_DELIMITER \| 39 | 40 | DIP_KILL_AT_THE_END true 41 | 42 | # Storage manager parameters 43 | # Storage directory for local runs 44 | STORAGE_LOCAL_DIR /tmp/ramdisk 45 | # Storage directory for cluster runs 46 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 47 | STORAGE_COLD_START true 48 | STORAGE_MEMORY_SIZE_MB 4096 49 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/low_selectivity/sample/local/eclipse-generic/0_01G_theta_line_self_join: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_lines_self_join 3 | DIP_SAMPLING true 4 | DIP_MATERIALIZED false 5 | DIP_TOPOLOGY_NAME_PREFIX username 6 | DIP_NUM_ACKERS 0 7 | 8 | # Define Input Paths Here: 9 | DIP_DATA_PATH test/data/tpch/0.01G/ 10 | DIP_RESULT_ROOT test/results/ 11 | DIP_KEY_REGION_ROOT test/m_bucket/key_region 12 | 13 | # Define Plan parameters for BCI 10G Z0 14 | DIP_SAMPLE_STATISTICS true 15 | FIRST_REL_SIZE 1384 16 | # 873000 17 | SECOND_REL_SIZE 51465 18 | # 51465000 19 | FIRST_NUM_OF_BUCKETS 50 20 | SECOND_NUM_OF_BUCKETS 50 21 | PAR_LAST_JOINERS 8 22 | 23 | # Define Parallelism Here: 24 | LINEITEM1_PAR 2 25 | LINEITEM2_PAR 2 26 | 27 | #below are unlikely to change 28 | DIP_EXTENSION .tbl 29 | DIP_READ_SPLIT_DELIMITER \| 30 | DIP_GLOBAL_ADD_DELIMITER | 31 | DIP_GLOBAL_SPLIT_DELIMITER \| 32 | 33 | DIP_KILL_AT_THE_END true 34 | # Storage manager parameters 35 | # Storage directory for local runs 36 | STORAGE_LOCAL_DIR /tmp/ramdisk 37 | # Storage directory for cluster runs 38 | STORAGE_CLUSTER_DIR /data/squall_zone/storage 39 | STORAGE_COLD_START true 40 | STORAGE_MEMORY_SIZE_MB 4096 41 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/config_template.txt: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED boolean(either true or false) 2 | DIP_QUERY_NAME String 3 | DIP_TOPOLOGY_NAME_PREFIX String 4 | DIP_TOPOLOGY_NAME String 5 | DIP_NUM_PARALLELISM int 6 | DIP_NUM_ACKERS int 7 | 8 | DIP_DATA_PATH String 9 | 10 | # for a component with name 'X' dedicate a line with: 11 | X_PAR int 12 | 13 | #below are unlikely to change 14 | DIP_EXTENSION String 15 | DIP_READ_SPLIT_DELIMITER String 16 | DIP_GLOBAL_ADD_DELIMITER String 17 | DIP_GLOBAL_SPLIT_DELIMITER String 18 | 19 | DIP_KILL_AT_THE_END true 20 | #used only in distributed mode 21 | DIP_NIMBUS_HOST icdatasrv2 22 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/formula/10G_hyracks_parallel: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX avitorovic 4 | DIP_TOPOLOGY_NAME 10G_hyracks_parallel 5 | 6 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 7 | #DIP_NUM_WORKERS 176 8 | #DIP_NUM_ACKERS 0 9 | 10 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/10G/ 11 | 12 | CUSTOMER_PAR 1 13 | ORDERS_PAR 3 14 | 15 | CUSTOMER_ORDERS_PAR 16 16 | 17 | #below are unlikely to change 18 | DIP_EXTENSION .tbl 19 | DIP_READ_SPLIT_DELIMITER \| 20 | DIP_GLOBAL_ADD_DELIMITER | 21 | DIP_GLOBAL_SPLIT_DELIMITER \| 22 | 23 | DIP_KILL_AT_THE_END true -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/formula/10K_rst_parallel: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME rst 3 | DIP_TOPOLOGY_NAME_PREFIX avitorovic 4 | DIP_TOPOLOGY_NAME rst_parallel 5 | 6 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 7 | #DIP_NUM_WORKERS 176 8 | #DIP_NUM_ACKERS 0 9 | 10 | DIP_DATA_PATH /export/home/avitorovic/queries/rst/data/10K/ 11 | 12 | DIP_EXTENSION .dat 13 | 14 | R_PAR 1 15 | S_PAR 1 16 | T_PAR 1 17 | 18 | R_S_PAR 16 19 | R_S_T_PAR 16 20 | 21 | # below are unlikely to change 22 | DIP_READ_SPLIT_DELIMITER , 23 | DIP_GLOBAL_ADD_DELIMITER | 24 | DIP_GLOBAL_SPLIT_DELIMITER \| 25 | 26 | DIP_KILL_AT_THE_END false -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/formula/1G_hyracksPreAgg_parallel: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracksPreAgg 3 | DIP_TOPOLOGY_NAME_PREFIX avitorovic 4 | DIP_TOPOLOGY_NAME 1G_hyracksPreAgg_parallel 5 | 6 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 7 | #DIP_NUM_WORKERS 176 8 | #DIP_NUM_ACKERS 0 9 | 10 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 11 | 12 | CUSTOMER_PAR 8 13 | ORDERS_PAR 8 14 | 15 | CUSTOMER_ORDERS_PAR 8 16 | 17 | #below are unlikely to change 18 | DIP_EXTENSION .tbl 19 | DIP_READ_SPLIT_DELIMITER \| 20 | DIP_GLOBAL_ADD_DELIMITER | 21 | DIP_GLOBAL_SPLIT_DELIMITER \| 22 | 23 | DIP_KILL_AT_THE_END true -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/formula/1G_hyracks_parallel: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME 1G_hyracks_parallel 5 | 6 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 7 | #DIP_NUM_WORKERS 176 8 | #DIP_NUM_ACKERS 0 9 | 10 | DIP_DATA_PATH /export/home/squalldata/tpchdb/1G 11 | 12 | CUSTOMER_PAR 8 13 | ORDERS_PAR 8 14 | 15 | CUSTOMER_ORDERS_PAR 8 16 | 17 | #below are unlikely to change 18 | DIP_EXTENSION .tbl 19 | DIP_READ_SPLIT_DELIMITER \| 20 | DIP_GLOBAL_ADD_DELIMITER | 21 | DIP_GLOBAL_SPLIT_DELIMITER \| 22 | 23 | DIP_KILL_AT_THE_END true 24 | 25 | # Storage manager parameters 26 | # Storage directory for local runs 27 | STORAGE_LOCAL_DIR /tmp/ramdisk 28 | # Storage directory for cluster runs 29 | STORAGE_DIP_DIR /export/home/squalldata/storage 30 | STORAGE_COLD_START true 31 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/formula/1G_tpch3_parallel_4: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch3 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME 1G_tpch3_parallel_4 5 | 6 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 7 | #DIP_NUM_WORKERS 176 8 | #DIP_NUM_ACKERS 0 9 | 10 | DIP_DATA_PATH /export/home/squalldata/tpchdb/1G 11 | 12 | CUSTOMER_PAR 4 13 | ORDERS_PAR 4 14 | LINEITEM_PAR 4 15 | 16 | CUSTOMER_ORDERS_PAR 2 17 | CUSTOMER_ORDERS_LINEITEM_PAR 4 18 | 19 | #below are unlikely to change 20 | DIP_EXTENSION .tbl 21 | DIP_READ_SPLIT_DELIMITER \| 22 | DIP_GLOBAL_ADD_DELIMITER | 23 | DIP_GLOBAL_SPLIT_DELIMITER \| 24 | 25 | DIP_KILL_AT_THE_END true 26 | 27 | # Storage manager parameters 28 | # Storage directory for local runs 29 | STORAGE_LOCAL_DIR /tmp/ramdisk 30 | # Storage directory for cluster runs 31 | STORAGE_DIP_DIR /export/home/squalldata/storage 32 | STORAGE_COLD_START true 33 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/formula/1G_tpch7_parallel_4: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch7 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_4 5 | 6 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 7 | #DIP_NUM_WORKERS 176 8 | #DIP_NUM_ACKERS 0 9 | 10 | DIP_DATA_PATH /export/home/squalldata/tpchdb/1G 11 | 12 | NATION1_PAR 1 13 | NATION2_PAR 1 14 | CUSTOMER_PAR 4 15 | ORDERS_PAR 4 16 | SUPPLIER_PAR 4 17 | LINEITEM_PAR 4 18 | 19 | NATION2_CUSTOMER_PAR 2 20 | NATION2_CUSTOMER_ORDERS_PAR 4 21 | SUPPLIER_NATION1_PAR 2 22 | LINEITEM_SUPPLIER_NATION1_PAR 4 23 | NATION2_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION1_PAR 8 24 | 25 | #below are unlikely to change 26 | DIP_EXTENSION .tbl 27 | DIP_READ_SPLIT_DELIMITER \| 28 | DIP_GLOBAL_ADD_DELIMITER | 29 | DIP_GLOBAL_SPLIT_DELIMITER \| 30 | 31 | DIP_KILL_AT_THE_END true 32 | 33 | # Storage manager parameters 34 | # Storage directory for local runs 35 | STORAGE_LOCAL_DIR /tmp/ramdisk 36 | # Storage directory for cluster runs 37 | STORAGE_DIP_DIR /export/home/squalldata/storage 38 | STORAGE_COLD_START true 39 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/formula/5G_hyracks_parallel: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX avitorovic 4 | DIP_TOPOLOGY_NAME 5G_hyracks_parallel 5 | 6 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 7 | #DIP_NUM_WORKERS 176 8 | #DIP_NUM_ACKERS 0 9 | 10 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/5G/ 11 | 12 | CUSTOMER_PAR 8 13 | ORDERS_PAR 8 14 | 15 | CUSTOMER_ORDERS_PAR 16 16 | #COUNTAGG_PAR 1 17 | 18 | #below are unlikely to change 19 | DIP_EXTENSION .tbl 20 | DIP_READ_SPLIT_DELIMITER \| 21 | DIP_GLOBAL_ADD_DELIMITER | 22 | DIP_GLOBAL_SPLIT_DELIMITER \| 23 | 24 | DIP_KILL_AT_THE_END true -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/formula/5G_hyracks_parallel_t2: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX avitorovic 4 | DIP_TOPOLOGY_NAME 5G_hyracks_parallel_t2 5 | 6 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 7 | #DIP_NUM_WORKERS 176 8 | #DIP_NUM_ACKERS 0 9 | 10 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/5G/ 11 | 12 | DIP_EXTENSION .tbl 13 | 14 | CUSTOMER_PAR 32 15 | ORDERS_PAR 32 16 | 17 | CUSTOMER_ORDERS_PAR 32 18 | 19 | # below are unlikely to change 20 | 21 | DIP_READ_SPLIT_DELIMITER \| 22 | DIP_GLOBAL_ADD_DELIMITER | 23 | DIP_GLOBAL_SPLIT_DELIMITER \| 24 | 25 | DIP_KILL_AT_THE_END true -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/formula/8G_tpch7_parallel_1: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED true 2 | DIP_QUERY_NAME tpch7 3 | DIP_TOPOLOGY_NAME_PREFIX avitorovic 4 | DIP_TOPOLOGY_NAME 8G_tpch7_parallel_4 5 | 6 | # the following two are optional, by default they use topology.workers and topology.ackers from storm.yaml 7 | #DIP_NUM_WORKERS 176 8 | #DIP_NUM_ACKERS 0 9 | 10 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/8G/ 11 | 12 | NATION1_PAR 1 13 | NATION2_PAR 1 14 | CUSTOMER_PAR 1 15 | ORDERS_PAR 1 16 | SUPPLIER_PAR 1 17 | LINEITEM_PAR 1 18 | 19 | NATION2_CUSTOMER_PAR 2 20 | NATION2_CUSTOMER_ORDERS_PAR 4 21 | SUPPLIER_NATION1_PAR 2 22 | LINEITEM_SUPPLIER_NATION1_PAR 4 23 | NATION2_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION1_PAR 8 24 | 25 | #below are unlikely to change 26 | DIP_EXTENSION .tbl 27 | DIP_READ_SPLIT_DELIMITER \| 28 | DIP_GLOBAL_ADD_DELIMITER | 29 | DIP_GLOBAL_SPLIT_DELIMITER \| 30 | 31 | DIP_KILL_AT_THE_END true 32 | 33 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_16_formula: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_16_formula 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 8 21 | ORDERS_PAR 12 22 | SUPPLIER_PAR 4 23 | LINEITEM_PAR 18 24 | 25 | NATION_CUSTOMER_PAR 4 26 | NATION_CUSTOMER_ORDERS_PAR 8 27 | SUPPLIER_NATION_PAR 4 28 | LINEITEM_SUPPLIER_NATION_PAR 8 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 16 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_16_opt: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_16_opt 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 10 21 | ORDERS_PAR 14 22 | SUPPLIER_PAR 8 23 | LINEITEM_PAR 18 24 | 25 | NATION_CUSTOMER_PAR 4 26 | NATION_CUSTOMER_ORDERS_PAR 8 27 | SUPPLIER_NATION_PAR 4 28 | LINEITEM_SUPPLIER_NATION_PAR 8 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 18 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_16_opt_32ACK: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_16_opt_32ACK 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 32 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 10 21 | ORDERS_PAR 14 22 | SUPPLIER_PAR 8 23 | LINEITEM_PAR 18 24 | 25 | NATION_CUSTOMER_PAR 4 26 | NATION_CUSTOMER_ORDERS_PAR 8 27 | SUPPLIER_NATION_PAR 4 28 | LINEITEM_SUPPLIER_NATION_PAR 8 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 18 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_1_20: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_1_20 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION1_PAR 1 20 | NATION2_PAR 1 21 | CUSTOMER_PAR 1 22 | ORDERS_PAR 1 23 | SUPPLIER_PAR 1 24 | LINEITEM_PAR 1 25 | 26 | NATION1_CUSTOMER_PAR 3 27 | NATION1_CUSTOMER_ORDERS_PAR 3 28 | SUPPLIER_NATION2_PAR 3 29 | LINEITEM_SUPPLIER_NATION2_PAR 3 30 | NATION1_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION2_PAR 7 31 | 32 | DIP_KILL_AT_THE_END true 33 | #used only in distributed mode 34 | DIP_NIMBUS_HOST icdatasrv2 35 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_1_20_F1: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_1_20_F1 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 1 21 | ORDERS_PAR 1 22 | SUPPLIER_PAR 1 23 | LINEITEM_PAR 1 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 3 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 3 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 1 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_1_20_F16: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_1_20_F16 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 1 21 | ORDERS_PAR 1 22 | SUPPLIER_PAR 1 23 | LINEITEM_PAR 1 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 3 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 3 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 16 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_1_20_F3: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_1_20_F3 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 1 21 | ORDERS_PAR 1 22 | SUPPLIER_PAR 1 23 | LINEITEM_PAR 1 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 3 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 3 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 3 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_1_2x: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_1_2x 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 1 21 | ORDERS_PAR 1 22 | SUPPLIER_PAR 1 23 | LINEITEM_PAR 1 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 4 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 4 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 9 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_2_20: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_2_20 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 2 21 | ORDERS_PAR 2 22 | SUPPLIER_PAR 2 23 | LINEITEM_PAR 2 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 3 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 3 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 7 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_2_20_8ACK: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_2_20_8ACK 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 8 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 2 21 | ORDERS_PAR 2 22 | SUPPLIER_PAR 2 23 | LINEITEM_PAR 2 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 3 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 3 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 7 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_2_20_BB: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_2_20_BB 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 2 21 | ORDERS_PAR 2 22 | SUPPLIER_PAR 2 23 | LINEITEM_PAR 2 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 3 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 3 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 7 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_2_20_SB: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_2_20_SB 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 2 21 | ORDERS_PAR 2 22 | SUPPLIER_PAR 2 23 | LINEITEM_PAR 2 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 3 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 3 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 7 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_2_20_formula: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_2_20_formula 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 2 21 | ORDERS_PAR 2 22 | SUPPLIER_PAR 2 23 | LINEITEM_PAR 2 24 | 25 | NATION_CUSTOMER_PAR 1 26 | NATION_CUSTOMER_ORDERS_PAR 2 27 | SUPPLIER_NATION_PAR 1 28 | LINEITEM_SUPPLIER_NATION_PAR 2 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 4 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_2_2x: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_2_2x 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 2 21 | ORDERS_PAR 2 22 | SUPPLIER_PAR 2 23 | LINEITEM_PAR 2 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 4 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 4 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 9 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_4_20: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_4_20 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 4 21 | ORDERS_PAR 4 22 | SUPPLIER_PAR 4 23 | LINEITEM_PAR 4 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 3 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 3 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 7 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_4_20_formula: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_4_20_formula 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 4 21 | ORDERS_PAR 4 22 | SUPPLIER_PAR 4 23 | LINEITEM_PAR 4 24 | 25 | NATION_CUSTOMER_PAR 2 26 | NATION_CUSTOMER_ORDERS_PAR 4 27 | SUPPLIER_NATION_PAR 2 28 | LINEITEM_SUPPLIER_NATION_PAR 4 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 8 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_4_2x: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_4_2x 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 4 21 | ORDERS_PAR 4 22 | SUPPLIER_PAR 4 23 | LINEITEM_PAR 4 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 4 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 4 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 9 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_4_2xx: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_4_2xx 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 4 21 | ORDERS_PAR 4 22 | SUPPLIER_PAR 4 23 | LINEITEM_PAR 4 24 | 25 | NATION_CUSTOMER_PAR 4 26 | NATION_CUSTOMER_ORDERS_PAR 8 27 | SUPPLIER_NATION_PAR 4 28 | LINEITEM_SUPPLIER_NATION_PAR 8 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 16 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_8_20: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_8_20 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 8 21 | ORDERS_PAR 8 22 | SUPPLIER_PAR 8 23 | LINEITEM_PAR 8 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 3 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 3 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 7 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_8_2EQ: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_8_2EQ 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 8 21 | ORDERS_PAR 8 22 | SUPPLIER_PAR 8 23 | LINEITEM_PAR 8 24 | 25 | NATION_CUSTOMER_PAR 8 26 | NATION_CUSTOMER_ORDERS_PAR 8 27 | SUPPLIER_NATION_PAR 8 28 | LINEITEM_SUPPLIER_NATION_PAR 8 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 16 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_8_2x: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_8_2x 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 8 21 | ORDERS_PAR 8 22 | SUPPLIER_PAR 8 23 | LINEITEM_PAR 8 24 | 25 | NATION_CUSTOMER_PAR 3 26 | NATION_CUSTOMER_ORDERS_PAR 4 27 | SUPPLIER_NATION_PAR 3 28 | LINEITEM_SUPPLIER_NATION_PAR 4 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 9 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_8_2xx: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_8_2xx 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 8 21 | ORDERS_PAR 8 22 | SUPPLIER_PAR 8 23 | LINEITEM_PAR 8 24 | 25 | NATION_CUSTOMER_PAR 4 26 | NATION_CUSTOMER_ORDERS_PAR 8 27 | SUPPLIER_NATION_PAR 4 28 | LINEITEM_SUPPLIER_NATION_PAR 8 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 16 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_8_2xxx: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_8_2xxx 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 8 21 | ORDERS_PAR 8 22 | SUPPLIER_PAR 8 23 | LINEITEM_PAR 8 24 | 25 | NATION_CUSTOMER_PAR 8 26 | NATION_CUSTOMER_ORDERS_PAR 16 27 | SUPPLIER_NATION_PAR 8 28 | LINEITEM_SUPPLIER_NATION_PAR 16 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 32 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_8_opt: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #TPCH7 query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME TPCH7 7 | DIP_TOPOLOGY_NAME 1G_tpch7_parallel_8_opt 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | DIP_READ_SPLIT_DELIMITER \| 16 | DIP_GLOBAL_ADD_DELIMITER | 17 | DIP_GLOBAL_SPLIT_DELIMITER \| 18 | 19 | NATION_PAR 1 20 | CUSTOMER_PAR 4 21 | ORDERS_PAR 6 22 | SUPPLIER_PAR 2 23 | LINEITEM_PAR 10 24 | 25 | NATION_CUSTOMER_PAR 4 26 | NATION_CUSTOMER_ORDERS_PAR 10 27 | SUPPLIER_NATION_PAR 4 28 | LINEITEM_SUPPLIER_NATION_PAR 10 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_PAR 16 30 | 31 | DIP_KILL_AT_THE_END true 32 | #used only in distributed mode 33 | DIP_NIMBUS_HOST icdatasrv2 34 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/1G_tpch7_parallel_old: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #RST query 5 | DIP_DISTRIBUTED true 6 | DIP_TOPOLOGY_NAME TPCH7-4816_Dst16 7 | DIP_NUM_PARALLELISM 100 8 | 9 | #DIP_DATA_PATH /home/vitorovi/working/installations/storm/data/tpch/1G/ 10 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/1G/ 11 | 12 | DIP_READ_SPLIT_DELIMITER \| 13 | DIP_GLOBAL_ADD_DELIMITER | 14 | DIP_GLOBAL_SPLIT_DELIMITER \| 15 | 16 | NATION_CUSTOMER_STORAGE_PAR 16 17 | NATION_CUSTOMER_MAT_PAR 2 18 | 19 | NATION_CUSTOMER_ORDERS_STORAGE_PAR 8 20 | NATION_CUSTOMER_ORDERS_MAT_PAR 4 21 | 22 | SUPPLIER_NATION_STORAGE_PAR 4 23 | SUPPLIER_NATION_MAT_PAR 2 24 | 25 | LINEITEM_SUPPLIER_NATION_STORAGE_PAR 8 26 | LINEITEM_SUPPLIER_NATION_MAT_PAR 4 27 | 28 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_STORAGE_PAR 16 29 | NATION_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION_MAT_PAR 1 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/4G_hyracks_parallel_t1: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #Hyracks query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME hyracks 7 | DIP_TOPOLOGY_NAME 4G_hyracks_parallel_t1 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/4G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | CUSTOMER_PAR 8 16 | ORDERS_PAR 8 17 | 18 | CUSTOMER_ORDERS_PAR 8 19 | COUNTAGG_PAR 5 20 | 21 | # below are unlikely to change 22 | 23 | DIP_READ_SPLIT_DELIMITER \| 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | #used only in distributed mode 29 | DIP_NIMBUS_HOST icdatasrv2 30 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/5G_hyracksPreAgg_parallel_t1: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #Hyracks query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME hyracksPreAgg 7 | DIP_TOPOLOGY_NAME 5G_hyracksPreAgg_parallel_t1 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/5G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | CUSTOMER_PAR 8 16 | ORDERS_PAR 8 17 | 18 | CUSTOMER_ORDERS_PAR 8 19 | COUNTAGG_PAR 5 20 | 21 | # below are unlikely to change 22 | 23 | DIP_READ_SPLIT_DELIMITER \| 24 | DIP_GLOBAL_ADD_DELIMITER | 25 | DIP_GLOBAL_SPLIT_DELIMITER \| 26 | 27 | DIP_KILL_AT_THE_END true 28 | #used only in distributed mode 29 | DIP_NIMBUS_HOST icdatasrv2 30 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/parallel/guess/5G_hyracksPreAgg_parallel_t2: -------------------------------------------------------------------------------- 1 | # To change this template, choose Tools | Templates 2 | # and open the template in the editor. 3 | 4 | #Hyracks query 5 | DIP_DISTRIBUTED true 6 | DIP_QUERY_NAME hyracksPreAgg 7 | DIP_TOPOLOGY_NAME 5G_hyracksPreAgg_parallel_t2 8 | DIP_NUM_PARALLELISM 176 9 | DIP_NUM_ACKERS 17 10 | 11 | DIP_DATA_PATH /export/home/avitorovic/queries/tpch/5G/ 12 | 13 | DIP_EXTENSION .tbl 14 | 15 | CUSTOMER_PAR 8 16 | ORDERS_PAR 8 17 | 18 | CUSTOMER_ORDERS_PAR 8 19 | 20 | # below are unlikely to change 21 | 22 | DIP_READ_SPLIT_DELIMITER \| 23 | DIP_GLOBAL_ADD_DELIMITER | 24 | DIP_GLOBAL_SPLIT_DELIMITER \| 25 | 26 | DIP_KILL_AT_THE_END true 27 | #used only in distributed mode 28 | DIP_NIMBUS_HOST icdatasrv2 29 | DIP_STORM_ZOOKEEPER_SERVERS icdatasrv2 -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_hyracks_pre_agg_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks_pre_agg 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME hyracksPreAgg_serial 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | DIP_EXTENSION .tbl 10 | 11 | CUSTOMER_PAR 1 12 | ORDERS_PAR 1 13 | 14 | CUSTOMER_ORDERS_PAR 1 15 | COUNTAGG_PAR 1 16 | 17 | # below are unlikely to change 18 | 19 | DIP_READ_SPLIT_DELIMITER \| 20 | DIP_GLOBAL_ADD_DELIMITER | 21 | DIP_GLOBAL_SPLIT_DELIMITER \| 22 | 23 | DIP_KILL_AT_THE_END true 24 | 25 | # Storage manager parameters 26 | # Storage directory for local runs 27 | STORAGE_LOCAL_DIR /tmp/ramdisk 28 | # Storage directory for cluster runs 29 | STORAGE_DIP_DIR /export/home/squalldata/storage 30 | STORAGE_COLD_START true 31 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_hyracks_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME hyracks_serial 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | CUSTOMER_PAR 1 10 | ORDERS_PAR 1 11 | 12 | CUSTOMER_ORDERS_PAR 1 13 | #COUNTAGG_PAR 1 14 | 15 | #below are unlikely to change 16 | DIP_EXTENSION .tbl 17 | DIP_READ_SPLIT_DELIMITER \| 18 | DIP_GLOBAL_ADD_DELIMITER | 19 | DIP_GLOBAL_SPLIT_DELIMITER \| 20 | 21 | DIP_KILL_AT_THE_END true 22 | 23 | # Storage manager parameters 24 | # Storage directory for local runs 25 | STORAGE_LOCAL_DIR /tmp/ramdisk 26 | # Storage directory for cluster runs 27 | STORAGE_DIP_DIR /export/home/squalldata/storage 28 | STORAGE_COLD_START true 29 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_theta_hyracks_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_hyracks 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME ThetaHyracks_serial 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | 10 | CUSTOMER_PAR 2 11 | ORDERS_PAR 2 12 | 13 | CUSTOMER_ORDERS_PAR 8 14 | COUNTAGG_PAR 1 15 | 16 | CUSTOMER_CARD 50 17 | ORDERS_CARD 50 18 | 19 | 20 | #below are unlikely to change 21 | DIP_EXTENSION .tbl 22 | DIP_READ_SPLIT_DELIMITER \| 23 | DIP_GLOBAL_ADD_DELIMITER | 24 | DIP_GLOBAL_SPLIT_DELIMITER \| 25 | 26 | DIP_KILL_AT_THE_END true 27 | 28 | # Storage manager parameters 29 | # Storage directory for local runs 30 | STORAGE_LOCAL_DIR /tmp/ramdisk 31 | # Storage directory for cluster runs 32 | STORAGE_DIP_DIR /export/home/squalldata/storage 33 | STORAGE_COLD_START true 34 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_theta_input_dominated_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_input_dominated 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME ThetaInputDominatedSimple 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | ORDERS_PAR 2 10 | LINEITEM_PAR 2 11 | 12 | LINEITEM_ORDERS_PAR 4 13 | 14 | #below are unlikely to change 15 | DIP_EXTENSION .tbl 16 | DIP_READ_SPLIT_DELIMITER \| 17 | DIP_GLOBAL_ADD_DELIMITER | 18 | DIP_GLOBAL_SPLIT_DELIMITER \| 19 | 20 | DIP_KILL_AT_THE_END true 21 | 22 | 23 | NATION_CARD 50 24 | CUSTOMER_CARD 50 25 | SUPPLIER_CARD 50 26 | ORDERS_CARD 50 27 | LINEITEM_CARD 50 28 | 29 | LINEITEM_ORDERS_CARD 50 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_DIP_DIR /export/home/squalldata/storage 36 | STORAGE_COLD_START true 37 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_theta_multiple_join_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_multiple_join 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME ThetaMultipleJoinSimple 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | ORDERS_PAR 2 10 | LINEITEM_PAR 2 11 | PARTSUPP_PAR 2 12 | SUPPLIER_PAR 2 13 | 14 | LINEITEM_ORDERS_PAR 2 15 | SUPPLIER_PARTSUPP_PAR 1 16 | LINEITEM_ORDERS_SUPPLIER_PARTSUPP_PAR 2 17 | 18 | #below are unlikely to change 19 | DIP_EXTENSION .tbl 20 | DIP_READ_SPLIT_DELIMITER \| 21 | DIP_GLOBAL_ADD_DELIMITER | 22 | DIP_GLOBAL_SPLIT_DELIMITER \| 23 | 24 | DIP_KILL_AT_THE_END true 25 | 26 | 27 | CUSTOMER_CARD 50 28 | SUPPLIER_CARD 50 29 | ORDERS_CARD 50 30 | LINEITEM_CARD 50 31 | PARTSUPP_CARD 50 32 | 33 | LINEITEM_ORDERS_CARD 50 34 | SUPPLIER_PARTSUPP_CARD 50 35 | 36 | # Storage manager parameters 37 | # Storage directory for local runs 38 | STORAGE_LOCAL_DIR /tmp/ramdisk 39 | # Storage directory for cluster runs 40 | STORAGE_DIP_DIR /export/home/squalldata/storage 41 | STORAGE_COLD_START true 42 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_theta_output_dominated_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME theta_output_dominated 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME ThetaOutputDominatedSimple 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | 10 | ORDERS_PAR 1 11 | LINEITEM_PAR 1 12 | NATION_PAR 1 13 | CUSTOMER_PAR 1 14 | SUPPLIER_PAR 1 15 | 16 | 17 | SUPPLIER_NATION_PAR 1 18 | 19 | #below are unlikely to change 20 | DIP_EXTENSION .tbl 21 | DIP_READ_SPLIT_DELIMITER \| 22 | DIP_GLOBAL_ADD_DELIMITER | 23 | DIP_GLOBAL_SPLIT_DELIMITER \| 24 | 25 | DIP_KILL_AT_THE_END true 26 | 27 | 28 | NATION_CARD 50 29 | CUSTOMER_CARD 50 30 | SUPPLIER_CARD 50 31 | ORDERS_CARD 50 32 | LINEITEM_CARD 50 33 | 34 | SUPPLIER_NATION_CARD 50 35 | 36 | # Storage manager parameters 37 | # Storage directory for local runs 38 | STORAGE_LOCAL_DIR /tmp/ramdisk 39 | # Storage directory for cluster runs 40 | STORAGE_DIP_DIR /export/home/squalldata/storage 41 | STORAGE_COLD_START true 42 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_tpch10_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch10 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME TPCH10Simple 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | CUSTOMER_PAR 1 10 | ORDERS_PAR 1 11 | NATION_PAR 1 12 | LINEITEM_PAR 1 13 | 14 | CUSTOMER_ORDERS_PAR 1 15 | CUSTOMER_ORDERS_NATION_PAR 1 16 | CUSTOMER_ORDERS_NATION_LINEITEM_PAR 1 17 | 18 | #below are unlikely to change 19 | DIP_EXTENSION .tbl 20 | DIP_READ_SPLIT_DELIMITER \| 21 | DIP_GLOBAL_ADD_DELIMITER | 22 | DIP_GLOBAL_SPLIT_DELIMITER \| 23 | 24 | DIP_KILL_AT_THE_END true 25 | 26 | # Storage manager parameters 27 | # Storage directory for local runs 28 | STORAGE_LOCAL_DIR /tmp/ramdisk 29 | # Storage directory for cluster runs 30 | STORAGE_DIP_DIR /export/home/squalldata/storage 31 | STORAGE_COLD_START true 32 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_tpch3_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch3 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME TPCH3Simple 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | CUSTOMER_PAR 1 10 | ORDERS_PAR 1 11 | LINEITEM_PAR 1 12 | 13 | CUSTOMER_ORDERS_PAR 1 14 | CUSTOMER_ORDERS_LINEITEM_PAR 1 15 | 16 | #below are unlikely to change 17 | DIP_EXTENSION .tbl 18 | DIP_READ_SPLIT_DELIMITER \| 19 | DIP_GLOBAL_ADD_DELIMITER | 20 | DIP_GLOBAL_SPLIT_DELIMITER \| 21 | 22 | DIP_KILL_AT_THE_END true 23 | 24 | # Storage manager parameters 25 | # Storage directory for local runs 26 | STORAGE_LOCAL_DIR /tmp/ramdisk 27 | # Storage directory for cluster runs 28 | STORAGE_DIP_DIR /export/home/squalldata/storage 29 | STORAGE_COLD_START true 30 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_tpch4_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch4 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME TPCH4Simple 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | ORDERS_PAR 1 10 | LINEITEM_PAR 1 11 | 12 | ORDERS_LINEITEM_PAR 1 13 | FINAL_RESULT_PAR 1 14 | 15 | #below are unlikely to change 16 | DIP_EXTENSION .tbl 17 | DIP_READ_SPLIT_DELIMITER \| 18 | DIP_GLOBAL_ADD_DELIMITER | 19 | DIP_GLOBAL_SPLIT_DELIMITER \| 20 | 21 | DIP_KILL_AT_THE_END true 22 | 23 | # Storage manager parameters 24 | # Storage directory for local runs 25 | STORAGE_LOCAL_DIR /tmp/ramdisk 26 | # Storage directory for cluster runs 27 | STORAGE_DIP_DIR /export/home/squalldata/storage 28 | STORAGE_COLD_START true 29 | -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_tpch5_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch5 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME TPCH5Simple 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | REGION_PAR 1 10 | NATION_PAR 1 11 | SUPPLIER_PAR 1 12 | LINEITEM_PAR 1 13 | CUSTOMER_PAR 1 14 | ORDERS_PAR 1 15 | 16 | REGION_NATION_PAR 1 17 | REGION_NATION_SUPPLIER_PAR 1 18 | REGION_NATION_SUPPLIER_LINEITEM_PAR 1 19 | CUSTOMER_ORDERS_PAR 1 20 | REGION_NATION_SUPPLIER_LINEITEM_CUSTOMER_ORDERS_PAR 1 21 | FINAL_RESULT_PAR 1 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_DIP_DIR /export/home/squalldata/storage 36 | STORAGE_COLD_START true -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_tpch7_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch7 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME TPCH7Simple 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | NATION1_PAR 1 10 | NATION2_PAR 1 11 | CUSTOMER_PAR 1 12 | ORDERS_PAR 1 13 | SUPPLIER_PAR 1 14 | LINEITEM_PAR 1 15 | 16 | NATION2_CUSTOMER_PAR 1 17 | NATION2_CUSTOMER_ORDERS_PAR 1 18 | SUPPLIER_NATION1_PAR 1 19 | LINEITEM_SUPPLIER_NATION1_PAR 1 20 | NATION2_CUSTOMER_ORDERS_LINEITEM_SUPPLIER_NATION1_PAR 1 21 | 22 | #below are unlikely to change 23 | DIP_EXTENSION .tbl 24 | DIP_READ_SPLIT_DELIMITER \| 25 | DIP_GLOBAL_ADD_DELIMITER | 26 | DIP_GLOBAL_SPLIT_DELIMITER \| 27 | 28 | DIP_KILL_AT_THE_END true 29 | 30 | # Storage manager parameters 31 | # Storage directory for local runs 32 | STORAGE_LOCAL_DIR /tmp/ramdisk 33 | # Storage directory for cluster runs 34 | STORAGE_DIP_DIR /export/home/squalldata/storage 35 | STORAGE_COLD_START true -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_tpch8_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch8 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME TPCH8Simple 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | REGION_PAR 1 10 | NATION1_PAR 1 11 | NATION2_PAR 1 12 | CUSTOMER_PAR 1 13 | SUPPLIER_PAR 1 14 | PART_PAR 1 15 | LINEITEM_PAR 1 16 | ORDERS_PAR 1 17 | 18 | REGION_NATION1_PAR 1 19 | REGION_NATION1_CUSTOMER_PAR 1 20 | 21 | SUPPLIER_NATION2_PAR 1 22 | 23 | PART_LINEITEM_PAR 1 24 | PART_LINEITEM_ORDERS_PAR 1 25 | SUPPLIER_NATION2_PART_LINEITEM_ORDERS_PAR 1 26 | 27 | REGION_NATION1_CUSTOMER_SUPPLIER_NATION2_PART_LINEITEM_ORDERS_PAR 1 28 | 29 | #below are unlikely to change 30 | DIP_EXTENSION .tbl 31 | DIP_READ_SPLIT_DELIMITER \| 32 | DIP_GLOBAL_ADD_DELIMITER | 33 | DIP_GLOBAL_SPLIT_DELIMITER \| 34 | 35 | DIP_KILL_AT_THE_END true 36 | 37 | # Storage manager parameters 38 | # Storage directory for local runs 39 | STORAGE_LOCAL_DIR /tmp/ramdisk 40 | # Storage directory for cluster runs 41 | STORAGE_DIP_DIR /export/home/squalldata/storage 42 | STORAGE_COLD_START true -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/0.01G_tpch9_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME tpch9 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME TPCH9Simple 5 | DIP_NUM_ACKERS 0 6 | 7 | DIP_DATA_PATH ../testing/data/tpch/0.01G/ 8 | 9 | 10 | PART_PAR 1 11 | LINEITEM_PAR 1 12 | PARTSUPP_PAR 1 13 | ORDERS_PAR 1 14 | SUPPLIER_PAR 1 15 | NATION_PAR 1 16 | 17 | PART_LINEITEM_PAR 1 18 | PART_LINEITEM_PARTSUPP_PAR 1 19 | PART_LINEITEM_PARTSUPP_ORDERS_PAR 1 20 | PART_LINEITEM_PARTSUPP_ORDERS_SUPPLIER_PAR 1 21 | PART_LINEITEM_PARTSUPP_ORDERS_SUPPLIER_NATION_PAR 1 22 | 23 | #below are unlikely to change 24 | DIP_EXTENSION .tbl 25 | DIP_READ_SPLIT_DELIMITER \| 26 | DIP_GLOBAL_ADD_DELIMITER | 27 | DIP_GLOBAL_SPLIT_DELIMITER \| 28 | 29 | DIP_KILL_AT_THE_END true 30 | 31 | # Storage manager parameters 32 | # Storage directory for local runs 33 | STORAGE_LOCAL_DIR /tmp/ramdisk 34 | # Storage directory for cluster runs 35 | STORAGE_DIP_DIR /export/home/squalldata/storage 36 | STORAGE_COLD_START true -------------------------------------------------------------------------------- /test/squall_plan_runner/confs/manual_batching/serial/10K_rst_serial: -------------------------------------------------------------------------------- 1 | DIP_DISTRIBUTED false 2 | DIP_QUERY_NAME rst 3 | DIP_TOPOLOGY_NAME_PREFIX username 4 | DIP_TOPOLOGY_NAME rst 5 | DIP_NUM_ACKERS 1 6 | 7 | DIP_DATA_PATH ../testing/data/rst/10K/ 8 | 9 | DIP_EXTENSION .dat 10 | 11 | R_PAR 1 12 | S_PAR 1 13 | T_PAR 1 14 | 15 | R_S_PAR 1 16 | R_S_T_PAR 1 17 | 18 | # below are unlikely to change 19 | DIP_READ_SPLIT_DELIMITER , 20 | DIP_GLOBAL_ADD_DELIMITER | 21 | DIP_GLOBAL_SPLIT_DELIMITER \| 22 | 23 | DIP_KILL_AT_THE_END true 24 | 25 | # Storage manager parameters 26 | # Storage directory for local runs 27 | STORAGE_LOCAL_DIR /tmp/ramdisk 28 | # Storage directory for cluster runs 29 | STORAGE_DIP_DIR /export/home/squalldata/storage 30 | STORAGE_COLD_START true --------------------------------------------------------------------------------