From 2aae2cb109c2cf1bfa3a6eaa7f2a41c4e7335a17 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 21 Oct 2015 12:15:15 +0200 Subject: [PATCH 01/60] TemplateTap had been deprecated for a while, dropped now. Drop TemplateSource. (hoping that's right) --- .../com/twitter/scalding/TemplateSource.scala | 121 ------------------ 1 file changed, 121 deletions(-) delete mode 100644 scalding-core/src/main/scala/com/twitter/scalding/TemplateSource.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TemplateSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/TemplateSource.scala deleted file mode 100644 index 32ed32ed0b..0000000000 --- a/scalding-core/src/main/scala/com/twitter/scalding/TemplateSource.scala +++ /dev/null @@ -1,121 +0,0 @@ -/* -Copyright 2013 Inkling, Inc. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - -http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ -package com.twitter.scalding - -import cascading.tap.hadoop.Hfs -import cascading.tap.hadoop.{ TemplateTap => HTemplateTap } -import cascading.tap.local.FileTap -import cascading.tap.local.{ TemplateTap => LTemplateTap } -import cascading.tap.SinkMode -import cascading.tap.Tap -import cascading.tuple.Fields - -/** - * This is a base class for template based output sources - */ -abstract class TemplateSource extends SchemedSource with HfsTapProvider { - - // The root path of the templated output. - def basePath: String - // The template as a java Formatter string. e.g. %s/%s for a two part template. - def template: String - // The fields to apply to the template. - def pathFields: Fields = Fields.ALL - - /** - * Creates the template tap. - * - * @param readOrWrite Describes if this source is being read from or written to. - * @param mode The mode of the job. (implicit) - * - * @returns A cascading TemplateTap. - */ - override def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = { - readOrWrite match { - case Read => throw new InvalidSourceException("Cannot use TemplateSource for input") - case Write => { - mode match { - case Local(_) => { - val localTap = new FileTap(localScheme, basePath, sinkMode) - new LTemplateTap(localTap, template, pathFields) - } - case hdfsMode @ Hdfs(_, _) => { - val hfsTap = createHfsTap(hdfsScheme, basePath, sinkMode) - new HTemplateTap(hfsTap, template, pathFields) - } - case hdfsTest @ HadoopTest(_, _) => { - val hfsTap = createHfsTap(hdfsScheme, hdfsTest.getWritePathFor(this), sinkMode) - new HTemplateTap(hfsTap, template, pathFields) - } - case _ => TestTapFactory(this, hdfsScheme).createTap(readOrWrite) - } - } - } - } - - /** - * Validates the taps, makes sure there are no nulls as the path or template. - * - * @param mode The mode of the job. - */ - override def validateTaps(mode: Mode): Unit = { - if (basePath == null) { - throw new InvalidSourceException("basePath cannot be null for TemplateTap") - } else if (template == null) { - throw new InvalidSourceException("template cannot be null for TemplateTap") - } - } -} - -/** - * An implementation of TSV output, split over a template tap. - * - * @param basePath The root path for the output. - * @param template The java formatter style string to use as the template. e.g. %s/%s. - * @param pathFields The set of fields to apply to the path. - * @param writeHeader Flag to indicate that the header should be written to the file. - * @param sinkMode How to handle conflicts with existing output. - * @param fields The set of fields to apply to the output. - */ -case class TemplatedTsv( - override val basePath: String, - override val template: String, - override val pathFields: Fields = Fields.ALL, - override val writeHeader: Boolean = false, - override val sinkMode: SinkMode = SinkMode.REPLACE, - override val fields: Fields = Fields.ALL) - extends TemplateSource with DelimitedScheme - -/** - * An implementation of SequenceFile output, split over a template tap. - * - * @param basePath The root path for the output. - * @param template The java formatter style string to use as the template. e.g. %s/%s. - * @param sequenceFields The set of fields to use for the sequence file. - * @param pathFields The set of fields to apply to the path. - * @param sinkMode How to handle conflicts with existing output. - */ -case class TemplatedSequenceFile( - override val basePath: String, - override val template: String, - val sequenceFields: Fields = Fields.ALL, - override val pathFields: Fields = Fields.ALL, - override val sinkMode: SinkMode = SinkMode.REPLACE) - extends TemplateSource with SequenceFileScheme { - - override val fields = sequenceFields -} - From 873c6500267ffb5450b001223a44dfc1ebd1f7f0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 21 Oct 2015 12:15:31 +0200 Subject: [PATCH 02/60] a few .gitignores (test data) --- scalding-hadoop-test/.gitignore | 1 + scalding-parquet/.gitignore | 1 + scalding-thrift-macros/.gitignore | 1 + 3 files changed, 3 insertions(+) create mode 100644 scalding-hadoop-test/.gitignore create mode 100644 scalding-parquet/.gitignore create mode 100644 scalding-thrift-macros/.gitignore diff --git a/scalding-hadoop-test/.gitignore b/scalding-hadoop-test/.gitignore new file mode 100644 index 0000000000..0beb3e581c --- /dev/null +++ b/scalding-hadoop-test/.gitignore @@ -0,0 +1 @@ +build/test diff --git a/scalding-parquet/.gitignore b/scalding-parquet/.gitignore new file mode 100644 index 0000000000..0beb3e581c --- /dev/null +++ b/scalding-parquet/.gitignore @@ -0,0 +1 @@ +build/test diff --git a/scalding-thrift-macros/.gitignore b/scalding-thrift-macros/.gitignore new file mode 100644 index 0000000000..0beb3e581c --- /dev/null +++ b/scalding-thrift-macros/.gitignore @@ -0,0 +1 @@ +build/test From 69b8dec434cf7e2c8074db2f0a4ac8576548fdf6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 21 Oct 2015 12:16:54 +0200 Subject: [PATCH 03/60] Begin attempting the jump to Cascading 3.0.2 ! --- project/Build.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index faaec2e027..f92dcff5fe 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -52,13 +52,13 @@ object ScaldingBuild extends Build { scalaVersion := "2.11.7", - crossScalaVersions := Seq("2.10.6", "2.11.7"), + crossScalaVersions := Seq(/* "2.10.5", */ "2.11.7"), ScalariformKeys.preferences := formattingPreferences, - javacOptions ++= Seq("-source", "1.6", "-target", "1.6"), + javacOptions ++= Seq("-source", "1.8", "-target", "1.8"), - javacOptions in doc := Seq("-source", "1.6"), + javacOptions in doc := Seq("-source", "1.8"), libraryDependencies ++= Seq( "org.mockito" % "mockito-all" % "1.8.5" % "test", @@ -280,10 +280,10 @@ object ScaldingBuild extends Build { lazy val scaldingDate = module("date") lazy val cascadingVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "2.6.1") + System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.0.2") lazy val cascadingJDBCVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "2.6.0") + System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "3.0.0-wip-119") lazy val scaldingBenchmarks = module("benchmarks").settings( libraryDependencies ++= Seq( From 28600d162b9599d318b7fc695811f94daf84706f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 21 Oct 2015 12:18:38 +0200 Subject: [PATCH 04/60] One property disappeared, got renamed. Follow suit + @deprecated --- .../main/scala/com/twitter/scalding/Config.scala | 16 ++++++++++++---- .../scala/com/twitter/scalding/Operations.scala | 6 +++--- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 9fa95b0e2c..4952bef9ea 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -18,11 +18,12 @@ package com.twitter.scalding import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.io.serializer.{ Serialization => HSerialization } +import org.slf4j.{ Logger, LoggerFactory } import com.twitter.chill.{ ExternalizerCodec, ExternalizerInjection, Externalizer, KryoInstantiator } import com.twitter.chill.config.{ ScalaMapConfig, ConfiguredInstantiator } import com.twitter.bijection.{ Base64String, Injection } -import cascading.pipe.assembly.AggregateBy +import cascading.pipe.assembly.{ AggregateByProps, AggregateBy } import cascading.flow.{ FlowListener, FlowStepListener, FlowProps, FlowStepStrategy } import cascading.property.AppProps import cascading.tuple.collect.SpillableProps @@ -36,6 +37,7 @@ import scala.util.{ Failure, Success, Try } * This is a wrapper class on top of Map[String, String] */ trait Config extends Serializable { + @transient private lazy val logger: Logger = LoggerFactory.getLogger(this.getClass) import Config._ // get the constants def toMap: Map[String, String] @@ -101,15 +103,21 @@ trait Config extends Serializable { def setMapSpillThreshold(count: Int): Config = this + (SpillableProps.MAP_THRESHOLD -> count.toString) + @deprecated("deprecated in Cascading 2.7 and dropped in Cascading 3.0, use setMapSideAggregationCapacity", "cascading 2.7") + def setMapSideAggregationThreshold(count: Int): Config = { + logger.warn("Ignoring deprecated setMapSideAggregationThreshold") + this + } + /* * Used in map-side aggregation of associative operations (Semigroup/Monoid) * This controls how many keys are in an in-memory cache. If a significant * probability mass of the key-space is far bigger than this value, it * does not help much (and may hurt, so experiment with disabling to get - * the best results + * the best results) */ - def setMapSideAggregationThreshold(count: Int): Config = - this + (AggregateBy.AGGREGATE_BY_THRESHOLD -> count.toString) + def setMapSideAggregationCapacity(capacity: Int): Config = + this + (AggregateByProps.AGGREGATE_BY_CAPACITY -> capacity.toString) /** * Set this configuration option to require all grouping/cogrouping diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala index 578c76639c..4a3a4089d4 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala @@ -18,7 +18,7 @@ package com.twitter.scalding { import cascading.operation._ import cascading.tuple._ import cascading.flow._ - import cascading.pipe.assembly.AggregateBy + import cascading.pipe.assembly.{ AggregateByProps, AggregateBy } import com.twitter.chill.MeatLocker import scala.collection.JavaConverters._ @@ -99,7 +99,7 @@ package com.twitter.scalding { /** * An implementation of map-side combining which is appropriate for associative and commutative functions * If a cacheSize is given, it is used, else we query - * the config for cascading.aggregateby.threshold (standard cascading param for an equivalent case) + * the config for cascading.aggregateby.capacity (standard cascading param for an equivalent case) * else we use a default value of 100,000 * * This keeps a cache of keys up to the cache-size, summing values as keys collide @@ -134,7 +134,7 @@ package com.twitter.scalding { val boxedSemigroup = Externalizer(commutativeSemigroup) val DEFAULT_CACHE_SIZE = 100000 - val SIZE_CONFIG_KEY = AggregateBy.AGGREGATE_BY_THRESHOLD + val SIZE_CONFIG_KEY = AggregateByProps.AGGREGATE_BY_CAPACITY def cacheSize(fp: FlowProcess[_]): Int = cacheSize.orElse { From 97463cc5b9a4384361b656b72e4bd7b7f4d72912 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 21 Oct 2015 12:19:27 +0200 Subject: [PATCH 05/60] Tap-n-Scheme generic definition slightly changed upstream, follow. --- .../etsy/cascading/tap/local/LocalTap.java | 31 ++++++++++--------- .../com/twitter/maple/hbase/HBaseScheme.java | 12 +++---- .../com/twitter/maple/hbase/HBaseTap.java | 8 ++--- .../maple/hbase/HBaseTapCollector.java | 4 +-- .../com/twitter/maple/tap/MemorySinkTap.java | 3 +- .../twitter/maple/tap/MemorySourceTap.java | 14 ++++----- .../java/com/twitter/maple/tap/StdoutTap.java | 3 +- .../com/twitter/scalding/FileSource.scala | 24 +++++++------- .../scalding/HfsConfPropertySetter.scala | 9 +++--- .../scala/com/twitter/scalding/Source.scala | 10 +++--- .../com/twitter/scalding/TestTapFactory.scala | 7 +++-- 11 files changed, 66 insertions(+), 59 deletions(-) diff --git a/maple/src/main/java/com/etsy/cascading/tap/local/LocalTap.java b/maple/src/main/java/com/etsy/cascading/tap/local/LocalTap.java index 1aefbbb49f..e31d87a190 100644 --- a/maple/src/main/java/com/etsy/cascading/tap/local/LocalTap.java +++ b/maple/src/main/java/com/etsy/cascading/tap/local/LocalTap.java @@ -5,6 +5,7 @@ import java.util.Properties; import java.util.logging.Logger; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.RecordReader; @@ -45,18 +46,18 @@ public class LocalTap extends Tap scheme, + public LocalTap(String path, Scheme scheme, SinkMode sinkMode) { super(new LocalScheme(scheme), sinkMode); setup(path, scheme); } - public LocalTap(String path, Scheme scheme) { + public LocalTap(String path, Scheme scheme) { super(new LocalScheme(scheme)); setup(path, scheme); } - private void setup(String path, Scheme scheme) { + private void setup(String path, Scheme scheme) { this.path = path; /* @@ -90,13 +91,13 @@ public String getIdentifier() { } @Override - public TupleEntryIterator openForRead(FlowProcess flowProcess, RecordReader input) throws IOException { + public TupleEntryIterator openForRead(FlowProcess flowProcess, RecordReader input) throws IOException { JobConf jobConf = mergeDefaults("LocalTap#openForRead", flowProcess.getConfigCopy(), defaults); return lfs.openForRead(new HadoopFlowProcess(jobConf)); } @Override - public TupleEntryCollector openForWrite(FlowProcess flowProcess, OutputCollector output) + public TupleEntryCollector openForWrite(FlowProcess flowProcess, OutputCollector output) throws IOException { JobConf jobConf = mergeDefaults("LocalTap#openForWrite", flowProcess.getConfigCopy(), defaults); return lfs.openForWrite(new HadoopFlowProcess(jobConf)); @@ -141,11 +142,11 @@ private static class LocalScheme extends Scheme { private static final long serialVersionUID = 5710119342340369543L; - private Scheme scheme; + private Scheme scheme; private JobConf defaults; private Lfs lfs; - public LocalScheme(Scheme scheme) { + public LocalScheme(Scheme scheme) { super(scheme.getSourceFields(), scheme.getSinkFields()); this.scheme = scheme; } @@ -159,19 +160,19 @@ private void setLfs(Lfs lfs) { } @Override - public Fields retrieveSourceFields(FlowProcess flowProcess, + public Fields retrieveSourceFields(FlowProcess flowProcess, Tap tap) { return scheme.retrieveSourceFields(new HadoopFlowProcess(defaults), lfs); } @Override - public void presentSourceFields(FlowProcess flowProcess, + public void presentSourceFields(FlowProcess flowProcess, Tap tap, Fields fields) { scheme.presentSourceFields(new HadoopFlowProcess(defaults), lfs, fields); } @Override - public void sourceConfInit(FlowProcess flowProcess, + public void sourceConfInit(FlowProcess flowProcess, Tap tap, Properties conf) { JobConf jobConf = mergeDefaults("LocalScheme#sourceConfInit", conf, defaults); scheme.sourceConfInit(new HadoopFlowProcess(jobConf), lfs, jobConf); @@ -179,19 +180,19 @@ public void sourceConfInit(FlowProcess flowProcess, } @Override - public Fields retrieveSinkFields(FlowProcess flowProcess, + public Fields retrieveSinkFields(FlowProcess flowProcess, Tap tap) { return scheme.retrieveSinkFields(new HadoopFlowProcess(defaults), lfs); } @Override - public void presentSinkFields(FlowProcess flowProcess, + public void presentSinkFields(FlowProcess flowProcess, Tap tap, Fields fields) { scheme.presentSinkFields(new HadoopFlowProcess(defaults), lfs, fields); } @Override - public void sinkConfInit(FlowProcess flowProcess, + public void sinkConfInit(FlowProcess flowProcess, Tap tap, Properties conf) { JobConf jobConf = mergeDefaults("LocalScheme#sinkConfInit", conf, defaults); scheme.sinkConfInit(new HadoopFlowProcess(jobConf), lfs, jobConf); @@ -199,13 +200,13 @@ public void sinkConfInit(FlowProcess flowProcess, } @Override - public boolean source(FlowProcess flowProcess, SourceCall sourceCall) + public boolean source(FlowProcess flowProcess, SourceCall sourceCall) throws IOException { throw new RuntimeException("LocalTap#source is never called"); } @Override - public void sink(FlowProcess flowProcess, SinkCall sinkCall) + public void sink(FlowProcess flowProcess, SinkCall sinkCall) throws IOException { throw new RuntimeException("LocalTap#sink is never called"); } diff --git a/maple/src/main/java/com/twitter/maple/hbase/HBaseScheme.java b/maple/src/main/java/com/twitter/maple/hbase/HBaseScheme.java index 0f830ede86..6dfa4ff7ff 100644 --- a/maple/src/main/java/com/twitter/maple/hbase/HBaseScheme.java +++ b/maple/src/main/java/com/twitter/maple/hbase/HBaseScheme.java @@ -154,7 +154,7 @@ public String[] getFamilyNames() { } @Override - public void sourcePrepare(FlowProcess flowProcess, + public void sourcePrepare(FlowProcess flowProcess, SourceCall sourceCall) { Object[] pair = new Object[]{sourceCall.getInput().createKey(), sourceCall.getInput().createValue()}; @@ -163,13 +163,13 @@ public void sourcePrepare(FlowProcess flowProcess, } @Override - public void sourceCleanup(FlowProcess flowProcess, + public void sourceCleanup(FlowProcess flowProcess, SourceCall sourceCall) { sourceCall.setContext(null); } @Override - public boolean source(FlowProcess flowProcess, + public boolean source(FlowProcess flowProcess, SourceCall sourceCall) throws IOException { Tuple result = new Tuple(); @@ -206,7 +206,7 @@ public boolean source(FlowProcess flowProcess, } @Override - public void sink(FlowProcess flowProcess, SinkCall sinkCall) + public void sink(FlowProcess flowProcess, SinkCall sinkCall) throws IOException { TupleEntry tupleEntry = sinkCall.getOutgoingEntry(); OutputCollector outputCollector = sinkCall.getOutput(); @@ -231,7 +231,7 @@ public void sink(FlowProcess flowProcess, SinkCall process, + public void sinkConfInit(FlowProcess process, Tap tap, JobConf conf) { conf.setOutputFormat(TableOutputFormat.class); @@ -240,7 +240,7 @@ public void sinkConfInit(FlowProcess process, } @Override - public void sourceConfInit(FlowProcess process, + public void sourceConfInit(FlowProcess process, Tap tap, JobConf conf) { conf.setInputFormat(TableInputFormat.class); diff --git a/maple/src/main/java/com/twitter/maple/hbase/HBaseTap.java b/maple/src/main/java/com/twitter/maple/hbase/HBaseTap.java index 37ebfb0a8e..cc5f764215 100644 --- a/maple/src/main/java/com/twitter/maple/hbase/HBaseTap.java +++ b/maple/src/main/java/com/twitter/maple/hbase/HBaseTap.java @@ -143,7 +143,7 @@ protected HBaseAdmin getHBaseAdmin(JobConf conf) throws MasterNotRunningExceptio } @Override - public void sinkConfInit(FlowProcess process, JobConf conf) { + public void sinkConfInit(FlowProcess process, JobConf conf) { if(quorumNames != null) { conf.set("hbase.zookeeper.quorum", quorumNames); } @@ -178,12 +178,12 @@ public String getIdentifier() { } @Override - public TupleEntryIterator openForRead(FlowProcess jobConfFlowProcess, RecordReader recordReader) throws IOException { + public TupleEntryIterator openForRead(FlowProcess jobConfFlowProcess, RecordReader recordReader) throws IOException { return new HadoopTupleEntrySchemeIterator(jobConfFlowProcess, this, recordReader); } @Override - public TupleEntryCollector openForWrite(FlowProcess jobConfFlowProcess, OutputCollector outputCollector) throws IOException { + public TupleEntryCollector openForWrite(FlowProcess jobConfFlowProcess, OutputCollector outputCollector) throws IOException { HBaseTapCollector hBaseCollector = new HBaseTapCollector( jobConfFlowProcess, this ); hBaseCollector.prepare(); return hBaseCollector; @@ -230,7 +230,7 @@ public long getModifiedTime(JobConf jobConf) throws IOException { } @Override - public void sourceConfInit(FlowProcess process, JobConf conf) { + public void sourceConfInit(FlowProcess process, JobConf conf) { // a hack for MultiInputFormat to see that there is a child format FileInputFormat.setInputPaths( conf, getPath() ); diff --git a/maple/src/main/java/com/twitter/maple/hbase/HBaseTapCollector.java b/maple/src/main/java/com/twitter/maple/hbase/HBaseTapCollector.java index f5ad1ed2dd..1f726c465a 100644 --- a/maple/src/main/java/com/twitter/maple/hbase/HBaseTapCollector.java +++ b/maple/src/main/java/com/twitter/maple/hbase/HBaseTapCollector.java @@ -42,7 +42,7 @@ public class HBaseTapCollector extends TupleEntrySchemeCollector hadoopFlowProcess; + private final FlowProcess hadoopFlowProcess; /** Field tap */ private final Tap tap; /** Field reporter */ @@ -58,7 +58,7 @@ public class HBaseTapCollector extends TupleEntrySchemeCollector flowProcess, Tap tap) throws IOException { + public HBaseTapCollector(FlowProcess flowProcess, Tap tap) throws IOException { super(flowProcess, tap.getScheme()); this.hadoopFlowProcess = flowProcess; this.tap = tap; diff --git a/maple/src/main/java/com/twitter/maple/tap/MemorySinkTap.java b/maple/src/main/java/com/twitter/maple/tap/MemorySinkTap.java index 6b71b08b5b..243c9b0712 100644 --- a/maple/src/main/java/com/twitter/maple/tap/MemorySinkTap.java +++ b/maple/src/main/java/com/twitter/maple/tap/MemorySinkTap.java @@ -7,6 +7,7 @@ import cascading.tuple.Tuple; import cascading.tuple.TupleEntry; import cascading.tuple.TupleEntryIterator; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; import java.io.File; @@ -43,7 +44,7 @@ public static String getTempDir() { } @Override - public boolean commitResource(JobConf conf) throws java.io.IOException { + public boolean commitResource(Configuration conf) throws java.io.IOException { TupleEntryIterator it = new HadoopFlowProcess(conf).openTapForRead(this); boolean first_time = true; diff --git a/maple/src/main/java/com/twitter/maple/tap/MemorySourceTap.java b/maple/src/main/java/com/twitter/maple/tap/MemorySourceTap.java index 1d07de3a23..fdff677ff1 100644 --- a/maple/src/main/java/com/twitter/maple/tap/MemorySourceTap.java +++ b/maple/src/main/java/com/twitter/maple/tap/MemorySourceTap.java @@ -46,7 +46,7 @@ public List getTuples() { } @Override - public void sourceConfInit(FlowProcess flowProcess, + public void sourceConfInit(FlowProcess flowProcess, Tap, Void> tap, JobConf conf) { FileInputFormat.setInputPaths(conf, this.id); conf.setInputFormat(TupleMemoryInputFormat.class); @@ -54,13 +54,13 @@ public void sourceConfInit(FlowProcess flowProcess, } @Override - public void sinkConfInit(FlowProcess flowProcess, + public void sinkConfInit(FlowProcess flowProcess, Tap, Void> tap, JobConf conf) { throw new UnsupportedOperationException("Not supported yet."); } @Override - public void sourcePrepare( FlowProcess flowProcess, SourceCall flowProcess, SourceCall> sourceCall ) { sourceCall.setContext( new Object[ 2 ] ); @@ -69,7 +69,7 @@ public void sourcePrepare( FlowProcess flowProcess, SourceCall flowProcess, SourceCall flowProcess, SourceCall> sourceCall) throws IOException { TupleWrapper key = (TupleWrapper) sourceCall.getContext()[ 0 ]; NullWritable value = (NullWritable) sourceCall.getContext()[ 1 ]; @@ -84,13 +84,13 @@ public boolean source(FlowProcess flowProcess, SourceCall flowProcess, SourceCall flowProcess, SourceCall> sourceCall ) { sourceCall.setContext( null ); } @Override - public void sink(FlowProcess flowProcess, SinkCall sinkCall ) throws IOException { + public void sink(FlowProcess flowProcess, SinkCall sinkCall ) throws IOException { throw new UnsupportedOperationException("Not supported."); } @@ -127,7 +127,7 @@ public boolean equals(Object object) { } @Override - public TupleEntryIterator openForRead( FlowProcess flowProcess, RecordReader flowProcess, RecordReader input ) throws IOException { // input may be null when this method is called on the client side or cluster side when accumulating // for a HashJoin diff --git a/maple/src/main/java/com/twitter/maple/tap/StdoutTap.java b/maple/src/main/java/com/twitter/maple/tap/StdoutTap.java index 5c3f5f0b29..6975ec5643 100644 --- a/maple/src/main/java/com/twitter/maple/tap/StdoutTap.java +++ b/maple/src/main/java/com/twitter/maple/tap/StdoutTap.java @@ -5,6 +5,7 @@ import cascading.tap.hadoop.Lfs; import cascading.tuple.Fields; import cascading.tuple.TupleEntryIterator; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; import java.io.File; @@ -32,7 +33,7 @@ public static String getTempDir() { } @Override - public boolean commitResource(JobConf conf) throws java.io.IOException { + public boolean commitResource(Configuration conf) throws java.io.IOException { TupleEntryIterator it = new HadoopFlowProcess(conf).openTapForRead(this); System.out.println(""); System.out.println(""); diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index c3a661a2d2..cd369a3c94 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -53,7 +53,7 @@ abstract class SchemedSource extends Source { throw ModeException("Cascading local mode not supported for: " + toString) /** The scheme to use if the source is on hdfs. */ - def hdfsScheme: Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _] = + def hdfsScheme: Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], _, _] = throw ModeException("Cascading Hadoop mode not supported for: " + toString) // The mode to use for output taps determining how conflicts with existing output are handled. @@ -61,7 +61,7 @@ abstract class SchemedSource extends Source { } trait HfsTapProvider { - def createHfsTap(scheme: Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _], + def createHfsTap(scheme: Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], _, _], path: String, sinkMode: SinkMode): Hfs = new Hfs(scheme, path, sinkMode) @@ -69,8 +69,8 @@ trait HfsTapProvider { private[scalding] object CastFileTap { // The scala compiler has problems with the generics in Cascading - def apply(tap: FileTap): Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]] = - tap.asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] + def apply(tap: FileTap): Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]] = + tap.asInstanceOf[Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]]] } /** @@ -89,7 +89,7 @@ trait LocalSourceOverride extends SchemedSource { * @param sinkMode The mode for handling output conflicts. * @returns A tap. */ - def createLocalTap(sinkMode: SinkMode): Tap[JobConf, _, _] = { + def createLocalTap(sinkMode: SinkMode): Tap[Configuration, _, _] = { val taps = localPaths.map { p: String => CastFileTap(new FileTap(localScheme, p, sinkMode)) @@ -230,7 +230,7 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride with Hf } tryTtp match { - case Success(s) => s + case Success(s: Tap[_, _, _]) => s case Failure(e) => throw new java.lang.IllegalArgumentException(s"Failed to create tap for: $toString, with error: ${e.getMessage}", e) } } @@ -288,8 +288,8 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride with Hf } } - protected def createHdfsReadTap(hdfsMode: Hdfs): Tap[JobConf, _, _] = { - val taps: List[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] = + protected def createHdfsReadTap(hdfsMode: Hdfs): Tap[Configuration, _, _] = { + val taps: List[Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]]] = goodHdfsPaths(hdfsMode) .toList.map { path => CastHfsTap(createHfsTap(hdfsScheme, path, sinkMode)) } taps.size match { @@ -306,8 +306,8 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride with Hf } } -class ScaldingMultiSourceTap(taps: Seq[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]]) - extends MultiSourceTap[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]], JobConf, RecordReader[_, _]](taps: _*) { +class ScaldingMultiSourceTap(taps: Seq[Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]]]) + extends MultiSourceTap[Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]], Configuration, RecordReader[_, _]](taps: _*) { private final val randomId = UUID.randomUUID.toString override def getIdentifier() = randomId override def hashCode: Int = randomId.hashCode @@ -402,9 +402,9 @@ trait SuccessFileSource extends FileSource { * Put another way, this runs a Hadoop tap outside of Hadoop in the Cascading local mode */ trait LocalTapSource extends LocalSourceOverride { - override def createLocalTap(sinkMode: SinkMode): Tap[JobConf, _, _] = { + override def createLocalTap(sinkMode: SinkMode): Tap[Configuration, _, _] = { val taps = localPaths.map { p => - new LocalTap(p, hdfsScheme, sinkMode).asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] + new LocalTap(p, hdfsScheme, sinkMode).asInstanceOf[Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]]] }.toSeq taps match { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/HfsConfPropertySetter.scala b/scalding-core/src/main/scala/com/twitter/scalding/HfsConfPropertySetter.scala index 230378d31f..5c9fdf77ca 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/HfsConfPropertySetter.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/HfsConfPropertySetter.scala @@ -17,6 +17,7 @@ package com.twitter.scalding import cascading.tap.hadoop.Hfs import cascading.tap.SinkMode +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf import cascading.flow.FlowProcess import org.apache.hadoop.mapred.RecordReader @@ -24,10 +25,10 @@ import org.apache.hadoop.mapred.OutputCollector import cascading.scheme.Scheme private[scalding] class ConfPropertiesHfsTap(config: Config, - scheme: Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _], + scheme: Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], _, _], stringPath: String, sinkMode: SinkMode) extends Hfs(scheme, stringPath, sinkMode) { - override def sourceConfInit(process: FlowProcess[JobConf], conf: JobConf): Unit = { + override def sourceConfInit(process: FlowProcess[_ <: Configuration], conf: Configuration): Unit = { config.toMap.foreach { case (k, v) => conf.set(k, v) @@ -35,7 +36,7 @@ private[scalding] class ConfPropertiesHfsTap(config: Config, super.sourceConfInit(process, conf) } - override def sinkConfInit(process: FlowProcess[JobConf], conf: JobConf): Unit = { + override def sinkConfInit(process: FlowProcess[_ <: Configuration], conf: Configuration): Unit = { config.toMap.foreach { case (k, v) => conf.set(k, v) @@ -55,7 +56,7 @@ trait HfsConfPropertySetter extends HfsTapProvider { def tapConfig: Config = Config.empty override def createHfsTap( - scheme: Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _], + scheme: Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], _, _], path: String, sinkMode: SinkMode): Hfs = new ConfPropertiesHfsTap(tapConfig, scheme, path, sinkMode) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Source.scala b/scalding-core/src/main/scala/com/twitter/scalding/Source.scala index 094613235c..173099b609 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Source.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Source.scala @@ -27,6 +27,7 @@ import cascading.tap.{ Tap, SourceTap, SinkTap } import cascading.tuple.{ Fields, Tuple => CTuple, TupleEntry, TupleEntryCollector, TupleEntryIterator } import cascading.pipe.Pipe +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputCollector @@ -94,13 +95,13 @@ case object Write extends AccessMode object HadoopSchemeInstance { def apply(scheme: Scheme[_, _, _, _, _]) = - scheme.asInstanceOf[Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _]] + scheme.asInstanceOf[Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], _, _]] } object CastHfsTap { // The scala compiler has problems with the generics in Cascading - def apply(tap: Hfs): Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]] = - tap.asInstanceOf[Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]]] + def apply(tap: Hfs): Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]] = + tap.asInstanceOf[Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]]] } /** @@ -251,7 +252,8 @@ class NullTap[Config, Input, Output, SourceContext, SinkContext] SinkMode.UPDATE) { def getIdentifier = "nullTap" - def openForWrite(flowProcess: FlowProcess[Config], output: Output) = + + def openForWrite(flowProcess: FlowProcess[_ <: Config], output: Output) = new TupleEntryCollector { override def add(te: TupleEntry) {} override def add(t: CTuple) {} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/TestTapFactory.scala b/scalding-core/src/main/scala/com/twitter/scalding/TestTapFactory.scala index db3426a9ff..f50c5d125d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/TestTapFactory.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/TestTapFactory.scala @@ -25,6 +25,7 @@ import cascading.scheme.NullScheme import java.io.{ Serializable, InputStream, OutputStream } +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputCollector import org.apache.hadoop.mapred.RecordReader @@ -44,9 +45,9 @@ object TestTapFactory extends Serializable { override def sourceFields: Fields = fields override def sinkFields: Fields = fields } - def apply[A, B](src: Source, scheme: Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], A, B]): TestTapFactory = apply(src, scheme, SinkMode.REPLACE) + def apply[A, B](src: Source, scheme: Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], A, B]): TestTapFactory = apply(src, scheme, SinkMode.REPLACE) def apply[A, B](src: Source, - scheme: Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], A, B], sinkMode: SinkMode): TestTapFactory = + scheme: Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], A, B], sinkMode: SinkMode): TestTapFactory = new TestTapFactory(src, sinkMode) { override def hdfsScheme = Some(scheme) } } @@ -57,7 +58,7 @@ class TestTapFactory(src: Source, sinkMode: SinkMode) extends Serializable { def sinkFields: Fields = hdfsScheme.map { _.getSinkFields }.getOrElse(sys.error("No sinkFields defined")) - def hdfsScheme: Option[Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _]] = None + def hdfsScheme: Option[Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], _, _]] = None def createTap(readOrWrite: AccessMode)(implicit mode: Mode): Tap[_, _, _] = { mode match { From 7552303fa19e97cdc5edc0d215f0677a9461be4c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 21 Oct 2015 12:19:46 +0200 Subject: [PATCH 06/60] CascadingStats is now CascadingStats[_] --- .../src/main/scala/com/twitter/scalding/Execution.scala | 2 +- scalding-core/src/main/scala/com/twitter/scalding/Job.scala | 4 ++-- .../src/main/scala/com/twitter/scalding/JobStats.scala | 6 +++--- .../src/main/scala/com/twitter/scalding/JobTest.scala | 6 +++--- .../src/main/scala/com/twitter/scalding/Stats.scala | 4 ++-- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala index 5481e0a3aa..ad21d3e91b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Execution.scala @@ -803,7 +803,7 @@ object ExecutionCounters { * Just gets the counters from the CascadingStats and ignores * all the other fields present */ - def fromCascading(cs: cascading.stats.CascadingStats): ExecutionCounters = new ExecutionCounters { + def fromCascading(cs: cascading.stats.CascadingStats[_]): ExecutionCounters = new ExecutionCounters { import scala.collection.JavaConverters._ val keys = (for { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala index 787986e5a6..9fada4ac36 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -254,7 +254,7 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { FlowStateMap.clear(flowDef) } - protected def handleStats(statsData: CascadingStats) { + protected def handleStats(statsData: CascadingStats[_]) { scaldingCascadingStats = Some(statsData) // TODO: Why the two ways to do stats? Answer: jank-den. if (args.boolean("scalding.flowstats")) { @@ -281,7 +281,7 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { // This awful name is designed to avoid collision // with subclasses @transient - private[scalding] var scaldingCascadingStats: Option[CascadingStats] = None + private[scalding] var scaldingCascadingStats: Option[CascadingStats[_]] = None /** * Save the Flow object after a run to allow clients to inspect the job. diff --git a/scalding-core/src/main/scala/com/twitter/scalding/JobStats.scala b/scalding-core/src/main/scala/com/twitter/scalding/JobStats.scala index dac1f1a720..33584813a2 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/JobStats.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/JobStats.scala @@ -21,7 +21,7 @@ import cascading.stats.{ CascadeStats, CascadingStats, FlowStats } import scala.util.{ Failure, Try } object JobStats { - def apply(stats: CascadingStats): JobStats = { + def apply(stats: CascadingStats[_]): JobStats = { val m = statsMap(stats) new JobStats( stats match { @@ -30,14 +30,14 @@ object JobStats { }) } - private def counterMap(stats: CascadingStats): Map[String, Map[String, Long]] = + private def counterMap(stats: CascadingStats[_]): Map[String, Map[String, Long]] = stats.getCounterGroups.asScala.map { group => (group, stats.getCountersFor(group).asScala.map { counter => (counter, stats.getCounterValue(group, counter)) }.toMap) }.toMap - private def statsMap(stats: CascadingStats): Map[String, Any] = + private def statsMap(stats: CascadingStats[_]): Map[String, Any] = Map( "counters" -> counterMap(stats), "duration" -> stats.getDuration, diff --git a/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala b/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala index 148f3b0eb1..874f84b19b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/JobTest.scala @@ -59,7 +59,7 @@ object CascadeTest { class JobTest(cons: (Args) => Job) { private var argsMap = Map[String, List[String]]() private val callbacks = Buffer[() => Unit]() - private val statsCallbacks = Buffer[(CascadingStats) => Unit]() + private val statsCallbacks = Buffer[(CascadingStats[_]) => Unit]() // TODO: Switch the following maps and sets from Source to String keys // to guard for scala equality bugs private var sourceMap: (Source) => Option[Buffer[Tuple]] = { _ => None } @@ -124,13 +124,13 @@ class JobTest(cons: (Args) => Job) { // If this test is checking for multiple jobs chained by next, this only checks // for the counters in the final job's FlowStat. def counter(counter: String, group: String = Stats.ScaldingGroup)(op: Long => Unit) = { - statsCallbacks += ((stats: CascadingStats) => op(Stats.getCounterValue(counter, group)(stats))) + statsCallbacks += ((stats: CascadingStats[_]) => op(Stats.getCounterValue(counter, group)(stats))) this } // Used to check an assertion on all custom counters of a given scalding job. def counters(op: Map[String, Long] => Unit) = { - statsCallbacks += ((stats: CascadingStats) => op(Stats.getAllCustomCounters()(stats))) + statsCallbacks += ((stats: CascadingStats[_]) => op(Stats.getAllCustomCounters()(stats))) this } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala index 94871425e0..7eb6d45745 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Stats.scala @@ -62,11 +62,11 @@ object Stats { // When getting a counter value, cascadeStats takes precedence (if set) and // flowStats is used after that. Returns None if neither is defined. - def getCounterValue(key: StatKey)(implicit cascadingStats: CascadingStats): Long = + def getCounterValue(key: StatKey)(implicit cascadingStats: CascadingStats[_]): Long = cascadingStats.getCounterValue(key.group, key.counter) // Returns a map of all custom counter names and their counts. - def getAllCustomCounters()(implicit cascadingStats: CascadingStats): Map[String, Long] = { + def getAllCustomCounters()(implicit cascadingStats: CascadingStats[_]): Map[String, Long] = { val counts = for { counter <- cascadingStats.getCountersFor(ScaldingGroup).asScala value = getCounterValue(counter) From f8b5125649e0e9b1aa2346c316d2f436450d85e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 21 Oct 2015 12:20:30 +0200 Subject: [PATCH 07/60] FlowStep generic definition changed as well --- .../scala/com/twitter/scalding/reducer_estimation/Common.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index d433fa24fd..eb8ed048d4 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -35,7 +35,7 @@ object Common { case t => Seq(t) } - def unrollTaps(step: FlowStep[JobConf]): Seq[Tap[_, _, _]] = + def unrollTaps(step: FlowStep[_ <: JobConf]): Seq[Tap[_, _, _]] = unrollTaps(step.getSources.asScala.toSeq) /** From 9282448fe12c773e5441c75fde517af6153b1585 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 21 Oct 2015 12:21:08 +0200 Subject: [PATCH 08/60] (possibly reckless) attempt at telling Scala which macro interface we'd like to use (really unsure of what doing here) --- .../macros/impl/OrderedBufferableProviderImpl.scala | 6 +++--- .../impl/ordered_serialization/CompileTimeLengthTypes.scala | 2 +- .../macros/impl/ordered_serialization/TreeOrderedBuf.scala | 2 +- .../providers/TraversablesOrderedBuf.scala | 2 +- .../ordered_serialization/providers/UnitOrderedBuf.scala | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala index 41ae537db0..f36468e4a6 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala @@ -16,7 +16,7 @@ package com.twitter.scalding.serialization.macros.impl import scala.language.experimental.macros -import scala.reflect.macros.Context +import scala.reflect.macros.whitebox.Context import scala.util.Random import com.twitter.scalding.serialization.OrderedSerialization @@ -25,8 +25,8 @@ import com.twitter.scalding.serialization.macros.impl.ordered_serialization.prov object OrderedSerializationProviderImpl { def normalizedDispatcher(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { - case tpe if (!tpe.toString.contains(ImplicitOrderedBuf.macroMarker) && !(tpe.normalize == tpe)) => - buildDispatcher(tpe.normalize) + case tpe if (!tpe.toString.contains(ImplicitOrderedBuf.macroMarker) && !(tpe.dealias == tpe)) => + buildDispatcher(tpe.dealias) } def scaldingBasicDispatchers(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala index 7a2640c603..72ae960c4e 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala @@ -16,7 +16,7 @@ package com.twitter.scalding.serialization.macros.impl.ordered_serialization import scala.language.experimental.macros -import scala.reflect.macros.Context +import scala.reflect.macros.blackbox.Context sealed trait CompileTimeLengthTypes[C <: Context] { val ctx: C diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala index d95242256c..d0355ff6eb 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala @@ -19,7 +19,7 @@ import com.twitter.scalding._ import com.twitter.scalding.serialization.OrderedSerialization import com.twitter.scalding.serialization.JavaStreamEnrichments import java.io.InputStream -import scala.reflect.macros.Context +import scala.reflect.macros.blackbox.Context import scala.language.experimental.macros import scala.util.control.NonFatal diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala index b552645cb5..fafcb7582c 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala @@ -16,7 +16,7 @@ package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros -import scala.reflect.macros.Context +import scala.reflect.macros.whitebox.Context import java.io.InputStream import com.twitter.scalding._ diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala index e0cedb05a9..10116e0126 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala @@ -16,7 +16,7 @@ package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros -import scala.reflect.macros.Context +import scala.reflect.macros.blackbox.Context import com.twitter.scalding._ import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } From f37cc26d944e6364289dc13e132f84d61eb62c23 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Fri, 23 Oct 2015 11:00:23 +0200 Subject: [PATCH 09/60] Revert "(possibly reckless) attempt at telling Scala which macro interface we'd like to use" This reverts commit 92f6abc1e97fd9b64270e0099507075ea91cd29b. --- .../macros/impl/OrderedBufferableProviderImpl.scala | 6 +++--- .../impl/ordered_serialization/CompileTimeLengthTypes.scala | 2 +- .../macros/impl/ordered_serialization/TreeOrderedBuf.scala | 2 +- .../providers/TraversablesOrderedBuf.scala | 2 +- .../ordered_serialization/providers/UnitOrderedBuf.scala | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala index f36468e4a6..41ae537db0 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/OrderedBufferableProviderImpl.scala @@ -16,7 +16,7 @@ package com.twitter.scalding.serialization.macros.impl import scala.language.experimental.macros -import scala.reflect.macros.whitebox.Context +import scala.reflect.macros.Context import scala.util.Random import com.twitter.scalding.serialization.OrderedSerialization @@ -25,8 +25,8 @@ import com.twitter.scalding.serialization.macros.impl.ordered_serialization.prov object OrderedSerializationProviderImpl { def normalizedDispatcher(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { - case tpe if (!tpe.toString.contains(ImplicitOrderedBuf.macroMarker) && !(tpe.dealias == tpe)) => - buildDispatcher(tpe.dealias) + case tpe if (!tpe.toString.contains(ImplicitOrderedBuf.macroMarker) && !(tpe.normalize == tpe)) => + buildDispatcher(tpe.normalize) } def scaldingBasicDispatchers(c: Context)(buildDispatcher: => PartialFunction[c.Type, TreeOrderedBuf[c.type]]): PartialFunction[c.Type, TreeOrderedBuf[c.type]] = { diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala index 72ae960c4e..7a2640c603 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/CompileTimeLengthTypes.scala @@ -16,7 +16,7 @@ package com.twitter.scalding.serialization.macros.impl.ordered_serialization import scala.language.experimental.macros -import scala.reflect.macros.blackbox.Context +import scala.reflect.macros.Context sealed trait CompileTimeLengthTypes[C <: Context] { val ctx: C diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala index d0355ff6eb..d95242256c 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/TreeOrderedBuf.scala @@ -19,7 +19,7 @@ import com.twitter.scalding._ import com.twitter.scalding.serialization.OrderedSerialization import com.twitter.scalding.serialization.JavaStreamEnrichments import java.io.InputStream -import scala.reflect.macros.blackbox.Context +import scala.reflect.macros.Context import scala.language.experimental.macros import scala.util.control.NonFatal diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala index fafcb7582c..b552645cb5 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/TraversablesOrderedBuf.scala @@ -16,7 +16,7 @@ package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros -import scala.reflect.macros.whitebox.Context +import scala.reflect.macros.Context import java.io.InputStream import com.twitter.scalding._ diff --git a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala index 10116e0126..e0cedb05a9 100644 --- a/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala +++ b/scalding-serialization/src/main/scala/com/twitter/scalding/serialization/macros/impl/ordered_serialization/providers/UnitOrderedBuf.scala @@ -16,7 +16,7 @@ package com.twitter.scalding.serialization.macros.impl.ordered_serialization.providers import scala.language.experimental.macros -import scala.reflect.macros.blackbox.Context +import scala.reflect.macros.Context import com.twitter.scalding._ import com.twitter.scalding.serialization.macros.impl.ordered_serialization.{ CompileTimeLengthTypes, ProductLike, TreeOrderedBuf } From 9dc3c23cd5a3389e87db97b15beff2890ad98246 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Fri, 23 Oct 2015 17:43:31 +0200 Subject: [PATCH 10/60] =?UTF-8?q?Cascading=202=E2=86=923=20generic=20API?= =?UTF-8?q?=20changes?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Cyrille Chépélov (TP12) --- .../twitter/scalding/avro/AvroSource.scala | 5 ++-- .../commons/source/LzoGenericScheme.scala | 23 +++++++++++-------- .../scalding/commons/source/LzoTraits.scala | 2 +- .../commons/source/LzoTypedText.scala | 2 +- .../com/twitter/scalding/MemoryTap.scala | 4 ++-- .../parquet/scrooge/ParquetScroogeScheme.java | 4 ++-- .../scrooge/Parquet346ScroogeScheme.scala | 2 +- .../thrift/Parquet346TBaseScheme.scala | 2 +- .../scheme/TypedParquetTupleScheme.scala | 8 +++---- 9 files changed, 28 insertions(+), 24 deletions(-) diff --git a/scalding-avro/src/main/scala/com/twitter/scalding/avro/AvroSource.scala b/scalding-avro/src/main/scala/com/twitter/scalding/avro/AvroSource.scala index 366f11869f..9355d11afa 100644 --- a/scalding-avro/src/main/scala/com/twitter/scalding/avro/AvroSource.scala +++ b/scalding-avro/src/main/scala/com/twitter/scalding/avro/AvroSource.scala @@ -25,6 +25,7 @@ import java.io.InputStream import java.io.OutputStream import java.util.Properties import cascading.tuple.Fields +import org.apache.hadoop.conf.Configuration import collection.JavaConverters._ import org.apache.hadoop.mapred.{ OutputCollector, RecordReader, JobConf } @@ -32,7 +33,7 @@ trait UnpackedAvroFileScheme extends FileSource { def schema: Option[Schema] // HadoopSchemeInstance gives compile errors in 2.10 for some reason - override def hdfsScheme = (new AvroScheme(schema.getOrElse(null))).asInstanceOf[Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _]] + override def hdfsScheme = (new AvroScheme(schema.getOrElse(null))).asInstanceOf[Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], _, _]] override def localScheme = (new LAvroScheme(schema.getOrElse(null))).asInstanceOf[Scheme[Properties, InputStream, OutputStream, _, _]] @@ -42,7 +43,7 @@ trait PackedAvroFileScheme[T] extends FileSource { def schema: Schema // HadoopSchemeInstance gives compile errors for this in 2.10 for some reason - override def hdfsScheme = (new PackedAvroScheme[T](schema)).asInstanceOf[Scheme[JobConf, RecordReader[_, _], OutputCollector[_, _], _, _]] + override def hdfsScheme = (new PackedAvroScheme[T](schema)).asInstanceOf[Scheme[Configuration, RecordReader[_, _], OutputCollector[_, _], _, _]] override def localScheme = (new LPackedAvroScheme[T](schema)).asInstanceOf[Scheme[Properties, InputStream, OutputStream, _, _]] } diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala index 25a6c4e0cf..a81f97485e 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala @@ -16,11 +16,12 @@ limitations under the License. package com.twitter.scalding.commons.source +import scala.annotation.meta.param import scala.reflect.ClassTag import com.twitter.bijection._ import com.twitter.chill.Externalizer -import com.twitter.elephantbird.cascading2.scheme.LzoBinaryScheme +import com.twitter.elephantbird.cascading3.scheme.LzoBinaryScheme import com.twitter.elephantbird.mapreduce.input.combine.DelegateCombineFileInputFormat import com.twitter.elephantbird.mapreduce.io.{ BinaryConverter, GenericWritable } import com.twitter.elephantbird.mapreduce.input.{ BinaryConverterProvider, MultiInputFormat } @@ -97,7 +98,7 @@ object LzoGenericScheme { /** * From a Binary Converter passed in configure in the JobConf using of that by ElephantBird */ - def setConverter[M](conv: BinaryConverter[M], conf: JobConf, confKey: String, overrideConf: Boolean = false): Unit = { + def setConverter[M](conv: BinaryConverter[M], conf: Configuration, confKey: String, overrideConf: Boolean = false): Unit = { if ((conf.get(confKey) == null) || overrideConf) { val extern = Externalizer(conv) try { @@ -115,26 +116,28 @@ object LzoGenericScheme { * Generic scheme for data stored as lzo-compressed protobuf messages. * Serialization is performed using the supplied BinaryConverter. */ -class LzoGenericScheme[M](@transient conv: BinaryConverter[M], clazz: Class[M]) extends LzoBinaryScheme[M, GenericWritable[M]] { +class LzoGenericScheme[M](@(transient @param) conv: BinaryConverter[M], clazz: Class[M]) extends LzoBinaryScheme[M, GenericWritable[M]] { + + val convBox = Externalizer(conv) override protected def prepareBinaryWritable(): GenericWritable[M] = - new GenericWritable(conv) + new GenericWritable(convBox.get) - override def sourceConfInit(fp: FlowProcess[JobConf], + override def sourceConfInit(fp: FlowProcess[_ <: Configuration], tap: Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]], conf: JobConf): Unit = { - LzoGenericScheme.setConverter(conv, conf, SourceConfigBinaryConverterProvider.ProviderConfKey) + LzoGenericScheme.setConverter(convBox.get, conf, SourceConfigBinaryConverterProvider.ProviderConfKey) MultiInputFormat.setClassConf(clazz, conf) MultiInputFormat.setGenericConverterClassConf(classOf[SourceConfigBinaryConverterProvider[_]], conf) DelegateCombineFileInputFormat.setDelegateInputFormat(conf, classOf[MultiInputFormat[_]]) } - override def sinkConfInit(fp: FlowProcess[JobConf], - tap: Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]], - conf: JobConf): Unit = { - LzoGenericScheme.setConverter(conv, conf, SinkConfigBinaryConverterProvider.ProviderConfKey) + override def sinkConfInit(fp: FlowProcess[_ <: Configuration], + tap: Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]], + conf: Configuration): Unit = { + LzoGenericScheme.setConverter(convBox.get, conf, SinkConfigBinaryConverterProvider.ProviderConfKey) LzoGenericBlockOutputFormat.setClassConf(clazz, conf) LzoGenericBlockOutputFormat.setGenericConverterClassConf(classOf[SinkConfigBinaryConverterProvider[_]], conf) DeprecatedOutputFormatWrapper.setOutputFormat(classOf[LzoGenericBlockOutputFormat[_]], conf) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTraits.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTraits.scala index eeb28fc929..def9bc1673 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTraits.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTraits.scala @@ -22,7 +22,7 @@ import cascading.scheme.Scheme import org.apache.thrift.TBase import com.google.protobuf.Message import com.twitter.bijection.Injection -import com.twitter.elephantbird.cascading2.scheme._ +import com.twitter.elephantbird.cascading3.scheme._ import com.twitter.scalding._ import com.twitter.scalding.Dsl._ import com.twitter.scalding.source.{ CheckedInversion, MaxFailuresCheck } diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTypedText.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTypedText.scala index 089968b9a4..83bc91907a 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTypedText.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoTypedText.scala @@ -3,7 +3,7 @@ package com.twitter.scalding.commons.source import cascading.scheme.Scheme import cascading.scheme.hadoop.{ TextDelimited => CHTextDelimited } import cascading.scheme.local.{ TextDelimited => CLTextDelimited } -import com.twitter.elephantbird.cascading2.scheme.LzoTextDelimited +import com.twitter.elephantbird.cascading3.scheme.LzoTextDelimited import com.twitter.scalding._ import com.twitter.scalding.source.TypedTextDelimited import com.twitter.scalding.source.TypedSep diff --git a/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala b/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala index 896c63496a..e754e784ab 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/MemoryTap.scala @@ -44,11 +44,11 @@ class MemoryTap[In, Out](val scheme: Scheme[Properties, In, Out, _, _], val tupl override def getModifiedTime(conf: Properties) = if (resourceExists(conf)) modifiedTime else 0L override lazy val getIdentifier: String = scala.math.random.toString - override def openForRead(flowProcess: FlowProcess[Properties], input: In) = { + override def openForRead(flowProcess: FlowProcess[_ <: Properties], input: In) = { new TupleEntryChainIterator(scheme.getSourceFields, tupleBuffer.toIterator.asJava) } - override def openForWrite(flowProcess: FlowProcess[Properties], output: Out): TupleEntryCollector = { + override def openForWrite(flowProcess: FlowProcess[_ <: Properties], output: Out): TupleEntryCollector = { tupleBuffer.clear new MemoryTupleEntryCollector(tupleBuffer, this) } diff --git a/scalding-parquet-scrooge/src/main/java/com/twitter/scalding/parquet/scrooge/ParquetScroogeScheme.java b/scalding-parquet-scrooge/src/main/java/com/twitter/scalding/parquet/scrooge/ParquetScroogeScheme.java index a4ef0bb2f6..39c902e41b 100644 --- a/scalding-parquet-scrooge/src/main/java/com/twitter/scalding/parquet/scrooge/ParquetScroogeScheme.java +++ b/scalding-parquet-scrooge/src/main/java/com/twitter/scalding/parquet/scrooge/ParquetScroogeScheme.java @@ -51,7 +51,7 @@ public ParquetScroogeScheme(ParquetValueScheme.Config config) { } @Override - public void sinkConfInit(FlowProcess fp, + public void sinkConfInit(FlowProcess fp, Tap tap, JobConf jobConf) { DeprecatedParquetOutputFormat.setAsOutputFormat(jobConf); ParquetOutputFormat.setWriteSupportClass(jobConf, ScroogeWriteSupport.class); @@ -59,7 +59,7 @@ public void sinkConfInit(FlowProcess fp, } @Override - public void sourceConfInit(FlowProcess fp, + public void sourceConfInit(FlowProcess fp, Tap tap, JobConf jobConf) { super.sourceConfInit(fp, tap, jobConf); jobConf.setInputFormat(DeprecatedParquetInputFormat.class); diff --git a/scalding-parquet-scrooge/src/main/scala/com/twitter/scalding/parquet/scrooge/Parquet346ScroogeScheme.scala b/scalding-parquet-scrooge/src/main/scala/com/twitter/scalding/parquet/scrooge/Parquet346ScroogeScheme.scala index fe1444f222..84b8aab370 100644 --- a/scalding-parquet-scrooge/src/main/scala/com/twitter/scalding/parquet/scrooge/Parquet346ScroogeScheme.scala +++ b/scalding-parquet-scrooge/src/main/scala/com/twitter/scalding/parquet/scrooge/Parquet346ScroogeScheme.scala @@ -31,7 +31,7 @@ import scala.util.control.NonFatal class Parquet346ScroogeScheme[T <: ThriftStruct](config: ParquetValueScheme.Config[T]) extends ParquetScroogeScheme[T](config) { - override def sourceConfInit(fp: FlowProcess[JobConf], + override def sourceConfInit(fp: FlowProcess[_ <: JobConf], tap: Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]], jobConf: JobConf): Unit = { diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/thrift/Parquet346TBaseScheme.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/thrift/Parquet346TBaseScheme.scala index 4e6a4f9235..c482f5bcf7 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/thrift/Parquet346TBaseScheme.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/thrift/Parquet346TBaseScheme.scala @@ -33,7 +33,7 @@ import scala.collection.JavaConverters._ class Parquet346TBaseScheme[T <: TBase[_, _]](config: ParquetValueScheme.Config[T]) extends ParquetTBaseScheme[T](config) { - override def sourceConfInit(fp: FlowProcess[JobConf], + override def sourceConfInit(fp: FlowProcess[_ <: JobConf], tap: Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]], jobConf: JobConf): Unit = { diff --git a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala index 3e06b8e107..f604dfb421 100644 --- a/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala +++ b/scalding-parquet/src/main/scala/com/twitter/scalding/parquet/tuple/scheme/TypedParquetTupleScheme.scala @@ -140,14 +140,14 @@ class TypedParquetTupleScheme[T](val readSupport: ParquetReadSupport[T], val wri type SourceCallType = SourceCall[Array[AnyRef], Reader] type SinkCallType = SinkCall[Array[AnyRef], Output] - override def sourceConfInit(flowProcess: FlowProcess[JobConf], tap: TapType, jobConf: JobConf): Unit = { + override def sourceConfInit(flowProcess: FlowProcess[_ <: JobConf], tap: TapType, jobConf: JobConf): Unit = { fp.map(ParquetInputFormat.setFilterPredicate(jobConf, _)) jobConf.setInputFormat(classOf[DeprecatedParquetInputFormat[T]]) jobConf.set(ParquetInputOutputFormat.READ_SUPPORT_INSTANCE, ParquetInputOutputFormat.injection(readSupport)) ParquetInputFormat.setReadSupportClass(jobConf, classOf[ReadSupportInstanceProxy[_]]) } - override def source(flowProcess: FlowProcess[JobConf], sc: SourceCallType): Boolean = { + override def source(flowProcess: FlowProcess[_ <: JobConf], sc: SourceCallType): Boolean = { val value: Container[T] = sc.getInput.createValue() val hasNext = sc.getInput.next(null, value) @@ -161,12 +161,12 @@ class TypedParquetTupleScheme[T](val readSupport: ParquetReadSupport[T], val wri } } - override def sinkConfInit(flowProcess: FlowProcess[JobConf], tap: TapType, jobConf: JobConf): Unit = { + override def sinkConfInit(flowProcess: FlowProcess[_ <: JobConf], tap: TapType, jobConf: JobConf): Unit = { jobConf.setOutputFormat(classOf[InnerDeprecatedParquetOutputFormat[T]]) jobConf.set(ParquetInputOutputFormat.WRITE_SUPPORT_INSTANCE, ParquetInputOutputFormat.injection(writeSupport)) } - override def sink(flowProcess: FlowProcess[JobConf], sinkCall: SinkCallType): Unit = { + override def sink(flowProcess: FlowProcess[_ <: JobConf], sinkCall: SinkCallType): Unit = { val tuple = sinkCall.getOutgoingEntry require(tuple.size == 1, "TypedParquetTupleScheme expects tuple with an arity of exactly 1, but found " + tuple.getFields) From 8a68c5c45ac1cbf5d78be573b7643facf01c32c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Fri, 23 Oct 2015 17:43:51 +0200 Subject: [PATCH 11/60] TemplateSource had to go, so should the test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Cyrille Chépélov (TP12) --- .../twitter/scalding/TemplateSourceTest.scala | 63 ------------------- 1 file changed, 63 deletions(-) delete mode 100644 scalding-core/src/test/scala/com/twitter/scalding/TemplateSourceTest.scala diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TemplateSourceTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TemplateSourceTest.scala deleted file mode 100644 index 366b5c6676..0000000000 --- a/scalding-core/src/test/scala/com/twitter/scalding/TemplateSourceTest.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* -Copyright 2013 Inkling, Inc. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - -http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package com.twitter.scalding - -import java.io.File -import scala.io.{ Source => ScalaSource } - -import org.scalatest.{ Matchers, WordSpec } - -class TemplateTestJob(args: Args) extends Job(args) { - try { - Tsv("input", ('col1, 'col2)).read.write(TemplatedTsv("base", "%s", 'col1)) - } catch { - case e: Exception => e.printStackTrace() - } -} - -class TemplateSourceTest extends WordSpec with Matchers { - import Dsl._ - "TemplatedTsv" should { - "split output by template" in { - val input = Seq(("A", 1), ("A", 2), ("B", 3)) - - // Need to save the job to allow, find the temporary directory data was written to - var job: Job = null; - def buildJob(args: Args): Job = { - job = new TemplateTestJob(args) - job - } - - JobTest(buildJob(_)) - .source(Tsv("input", ('col1, 'col2)), input) - .runHadoop - .finish - - val testMode = job.mode.asInstanceOf[HadoopTest] - - val directory = new File(testMode.getWritePathFor(TemplatedTsv("base", "%s", 'col1))) - - directory.listFiles().map({ _.getName() }).toSet shouldBe Set("A", "B") - - val aSource = ScalaSource.fromFile(new File(directory, "A/part-00000")) - val bSource = ScalaSource.fromFile(new File(directory, "B/part-00000")) - - aSource.getLines.toList shouldBe Seq("A\t1", "A\t2") - bSource.getLines.toList shouldBe Seq("B\t3") - } - } -} From 24f4fbe6d2dec75ad2400564c343446b05d9bd76 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Fri, 23 Oct 2015 17:44:12 +0200 Subject: [PATCH 12/60] Can't reference changed jgrapht internals anymore MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Cyrille Chépélov (TP12) --- .../main/scala/com/twitter/scalding/platform/LocalCluster.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala index d5ae569b50..37e03ad95e 100644 --- a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala @@ -135,7 +135,7 @@ class LocalCluster(mutex: Boolean = true) { classOf[com.twitter.chill.algebird.AveragedValueSerializer], classOf[com.twitter.algebird.Semigroup[_]], classOf[com.twitter.chill.KryoInstantiator], - classOf[org.jgrapht.ext.EdgeNameProvider[_]], + //classOf[org.jgrapht.ext.EdgeNameProvider[_]], classOf[org.apache.commons.lang.StringUtils], classOf[cascading.scheme.local.TextDelimited], classOf[org.apache.commons.logging.LogFactory], From 9275149037dd6aaee1382cef97c410ac3d49191f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Fri, 23 Oct 2015 17:44:26 +0200 Subject: [PATCH 13/60] steps no longer have numbers, they have an ID --- .../hraven/reducer_estimation/HRavenHistoryService.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala index 0e7b0ffa68..21d7cf3bfa 100644 --- a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala +++ b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala @@ -120,12 +120,12 @@ object HRavenHistoryService extends HistoryService { */ def fetchPastJobDetails(step: FlowStep[JobConf], max: Int): Try[Seq[JobDetails]] = { val conf = step.getConfig - val stepNum = step.getStepNum + val stepId = step.getID def findMatchingJobStep(pastFlow: Flow) = pastFlow.getJobs.asScala.find { step => try { - step.getConfiguration.get("cascading.flow.step.num").toInt == stepNum + step.getConfiguration.get("cascading.flow.step.id").toInt == stepId } catch { case _: NumberFormatException => false } From b3e1adc28aae9764786fbf53d590487ccda49fa0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Fri, 23 Oct 2015 17:45:38 +0200 Subject: [PATCH 14/60] (not sure about 2.10) scala complains about "useless" @transient declaration (probably not the best PR to do this, nor the best thing; >50% change I'll revert this anyway) --- .../commons/source/DailySources.scala | 4 +- .../commons/source/HourlySources.scala | 4 +- .../source/VersionedKeyValSource.scala | 3 +- .../com/twitter/scalding/Operations.scala | 43 ++++++++++--------- .../twitter/scalding/source/CodecSource.scala | 3 +- .../scalding/typed/BijectedSourceSink.scala | 4 +- .../twitter/scalding/typed/CoGrouped.scala | 5 ++- .../twitter/scalding/typed/TypedPipe.scala | 7 +-- 8 files changed, 42 insertions(+), 31 deletions(-) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/DailySources.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/DailySources.scala index 21fa7b7e35..c89e3158ad 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/DailySources.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/DailySources.scala @@ -26,10 +26,12 @@ import cascading.tuple.Fields import java.io.Serializable import org.apache.thrift.TBase +import scala.annotation.meta.param + // Retrieve implicits import Dsl._ -abstract class DailySuffixLzoCodec[T](prefix: String, dateRange: DateRange)(implicit @transient suppliedInjection: Injection[T, Array[Byte]]) +abstract class DailySuffixLzoCodec[T](prefix: String, dateRange: DateRange)(implicit @(transient @param) suppliedInjection: Injection[T, Array[Byte]]) extends DailySuffixSource(prefix, dateRange) with LzoCodec[T] { val boxed = Externalizer(suppliedInjection) override lazy val injection = boxed.get diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/HourlySources.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/HourlySources.scala index 568dce0609..1ecd671772 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/HourlySources.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/HourlySources.scala @@ -24,7 +24,9 @@ import com.twitter.scalding._ import com.twitter.scalding.source._ import org.apache.thrift.TBase -abstract class HourlySuffixLzoCodec[T](prefix: String, dateRange: DateRange)(implicit @transient suppliedInjection: Injection[T, Array[Byte]]) +import scala.annotation.meta.param + +abstract class HourlySuffixLzoCodec[T](prefix: String, dateRange: DateRange)(implicit @(transient @param) suppliedInjection: Injection[T, Array[Byte]]) extends HourlySuffixSource(prefix, dateRange) with LzoCodec[T] { val boxed = Externalizer(suppliedInjection) override lazy val injection = boxed.get diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala index 177bb3c416..7d3873dfc9 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/VersionedKeyValSource.scala @@ -33,6 +33,7 @@ import com.twitter.scalding.source.{ CheckedInversion, MaxFailuresCheck } import com.twitter.scalding.typed.KeyedListLike import com.twitter.scalding.typed.TypedSink import org.apache.hadoop.mapred.JobConf +import scala.annotation.meta.param import scala.collection.JavaConverters._ /** @@ -54,7 +55,7 @@ object VersionedKeyValSource { class VersionedKeyValSource[K, V](val path: String, val sourceVersion: Option[Long], val sinkVersion: Option[Long], val maxFailures: Int, val versionsToKeep: Int)( - implicit @transient codec: Injection[(K, V), (Array[Byte], Array[Byte])]) extends Source + implicit @(transient @param) codec: Injection[(K, V), (Array[Byte], Array[Byte])]) extends Source with Mappable[(K, V)] with TypedSink[(K, V)] { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala index 4a3a4089d4..8b28fcc011 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Operations.scala @@ -20,6 +20,7 @@ package com.twitter.scalding { import cascading.flow._ import cascading.pipe.assembly.{ AggregateByProps, AggregateBy } import com.twitter.chill.MeatLocker + import scala.annotation.meta.{ field, param } import scala.collection.JavaConverters._ import com.twitter.algebird.{ Semigroup, SummingCache } @@ -34,7 +35,7 @@ package com.twitter.scalding { } } - class FlatMapFunction[S, T](@transient fn: S => TraversableOnce[T], fields: Fields, + class FlatMapFunction[S, T](@(transient @param) fn: S => TraversableOnce[T], fields: Fields, conv: TupleConverter[S], set: TupleSetter[T]) extends BaseOperation[Any](fields) with Function[Any] with ScaldingPrepare[Any] { val lockedFn = Externalizer(fn) @@ -46,7 +47,7 @@ package com.twitter.scalding { } } - class MapFunction[S, T](@transient fn: S => T, fields: Fields, + class MapFunction[S, T](@(transient @param) fn: S => T, fields: Fields, conv: TupleConverter[S], set: TupleSetter[T]) extends BaseOperation[Any](fields) with Function[Any] with ScaldingPrepare[Any] { val lockedFn = Externalizer(fn) @@ -67,7 +68,7 @@ package com.twitter.scalding { } } - class CleanupIdentityFunction(@transient fn: () => Unit) + class CleanupIdentityFunction(@(transient @param) fn: () => Unit) extends BaseOperation[Any](Fields.ALL) with Function[Any] with ScaldingPrepare[Any] { val lockedEf = Externalizer(fn) @@ -80,7 +81,7 @@ package com.twitter.scalding { } } - class CollectFunction[S, T](@transient fn: PartialFunction[S, T], fields: Fields, + class CollectFunction[S, T](@(transient @param) fn: PartialFunction[S, T], fields: Fields, conv: TupleConverter[S], set: TupleSetter[T]) extends BaseOperation[Any](fields) with Function[Any] with ScaldingPrepare[Any] { @@ -124,7 +125,7 @@ package com.twitter.scalding { * the typed-API. */ class MapsideReduce[V]( - @transient commutativeSemigroup: Semigroup[V], + @(transient @param) commutativeSemigroup: Semigroup[V], keyFields: Fields, valueFields: Fields, cacheSize: Option[Int])(implicit conv: TupleConverter[V], set: TupleSetter[V]) extends BaseOperation[SummingCache[Tuple, V]](Fields.join(keyFields, valueFields)) @@ -194,8 +195,8 @@ package com.twitter.scalding { * BaseOperation with support for context */ abstract class SideEffectBaseOperation[C]( - @transient bf: => C, // begin function returns a context - @transient ef: C => Unit, // end function to clean up context object + @(transient @param) bf: => C, // begin function returns a context + @(transient @param) ef: C => Unit, // end function to clean up context object fields: Fields) extends BaseOperation[C](fields) with ScaldingPrepare[C] { val lockedBf = Externalizer(() => bf) val lockedEf = Externalizer(ef) @@ -213,7 +214,7 @@ package com.twitter.scalding { */ class SideEffectMapFunction[S, C, T]( bf: => C, // begin function returns a context - @transient fn: (C, S) => T, // function that takes a context and a tuple and generate a new tuple + @(transient @param) fn: (C, S) => T, // function that takes a context and a tuple and generate a new tuple ef: C => Unit, // end function to clean up context object fields: Fields, conv: TupleConverter[S], @@ -233,7 +234,7 @@ package com.twitter.scalding { */ class SideEffectFlatMapFunction[S, C, T]( bf: => C, // begin function returns a context - @transient fn: (C, S) => TraversableOnce[T], // function that takes a context and a tuple, returns TraversableOnce of T + @(transient @param) fn: (C, S) => TraversableOnce[T], // function that takes a context and a tuple, returns TraversableOnce of T ef: C => Unit, // end function to clean up context object fields: Fields, conv: TupleConverter[S], @@ -247,7 +248,7 @@ package com.twitter.scalding { } } - class FilterFunction[T](@transient fn: T => Boolean, conv: TupleConverter[T]) + class FilterFunction[T](@(transient @param) fn: T => Boolean, conv: TupleConverter[T]) extends BaseOperation[Any] with Filter[Any] with ScaldingPrepare[Any] { val lockedFn = Externalizer(fn) @@ -258,7 +259,7 @@ package com.twitter.scalding { // All the following are operations for use in GroupBuilder - class FoldAggregator[T, X](@transient fn: (X, T) => X, @transient init: X, fields: Fields, + class FoldAggregator[T, X](@(transient @param) fn: (X, T) => X, @(transient @param) init: X, fields: Fields, conv: TupleConverter[T], set: TupleSetter[X]) extends BaseOperation[X](fields) with Aggregator[X] with ScaldingPrepare[X] { val lockedFn = Externalizer(fn) @@ -288,9 +289,9 @@ package com.twitter.scalding { * fields are the declared fields of this aggregator */ class MRMAggregator[T, X, U]( - @transient inputFsmf: T => X, - @transient inputRfn: (X, X) => X, - @transient inputMrfn: X => U, + @(transient @param) inputFsmf: T => X, + @(transient @param) inputRfn: (X, X) => X, + @(transient @param) inputMrfn: X => U, fields: Fields, conv: TupleConverter[T], set: TupleSetter[U]) extends BaseOperation[Tuple](fields) with Aggregator[Tuple] with ScaldingPrepare[Tuple] { val fsmf = Externalizer(inputFsmf) @@ -388,8 +389,8 @@ package com.twitter.scalding { * style purity. */ class MRMFunctor[T, X]( - @transient inputMrfn: T => X, - @transient inputRfn: (X, X) => X, + @(transient @param) inputMrfn: T => X, + @(transient @param) inputRfn: (X, X) => X, fields: Fields, conv: TupleConverter[T], set: TupleSetter[X]) extends FoldFunctor[X](fields) { @@ -423,8 +424,8 @@ package com.twitter.scalding { new MRMAggregator[X, X, U](args => args, rfn, mfn2, declaredFields, midConv, endSet)) class BufferOp[I, T, X]( - @transient init: I, - @transient inputIterfn: (I, Iterator[T]) => TraversableOnce[X], + @(transient @param) init: I, + @(transient @param) inputIterfn: (I, Iterator[T]) => TraversableOnce[X], fields: Fields, conv: TupleConverter[T], set: TupleSetter[X]) extends BaseOperation[Any](fields) with Buffer[Any] with ScaldingPrepare[Any] { val iterfn = Externalizer(inputIterfn) @@ -442,9 +443,9 @@ package com.twitter.scalding { * A buffer that allows state object to be set up and tear down. */ class SideEffectBufferOp[I, T, C, X]( - @transient init: I, + @(transient @param) init: I, bf: => C, // begin function returns a context - @transient inputIterfn: (I, C, Iterator[T]) => TraversableOnce[X], + @(transient @param) inputIterfn: (I, C, Iterator[T]) => TraversableOnce[X], ef: C => Unit, // end function to clean up context object fields: Fields, conv: TupleConverter[T], @@ -480,7 +481,7 @@ package com.twitter.scalding { class TypedBufferOp[K, V, U]( conv: TupleConverter[K], convV: TupleConverter[V], - @transient reduceFn: (K, Iterator[V]) => Iterator[U], + @(transient @param) reduceFn: (K, Iterator[V]) => Iterator[U], valueField: Fields) extends BaseOperation[Any](valueField) with Buffer[Any] with ScaldingPrepare[Any] { val reduceFnSer = Externalizer(reduceFn) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala index 394d7b7d10..10d3b82a9a 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/source/CodecSource.scala @@ -26,6 +26,7 @@ import com.twitter.scalding._ import java.util.Arrays import org.apache.hadoop.io.BytesWritable +import scala.annotation.meta.param import scala.collection.JavaConverters._ /** @@ -46,7 +47,7 @@ object CodecSource { def apply[T](paths: String*)(implicit codec: Injection[T, Array[Byte]]) = new CodecSource[T](paths) } -class CodecSource[T] private (val hdfsPaths: Seq[String], val maxFailures: Int = 0)(implicit @transient injection: Injection[T, Array[Byte]]) +class CodecSource[T] private (val hdfsPaths: Seq[String], val maxFailures: Int = 0)(implicit @(transient @param) injection: Injection[T, Array[Byte]]) extends FileSource with Mappable[T] with LocalTapSource { diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/BijectedSourceSink.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/BijectedSourceSink.scala index 4bbe1b3807..c1270018f6 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/BijectedSourceSink.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/BijectedSourceSink.scala @@ -22,13 +22,15 @@ import com.twitter.bijection.ImplicitBijection import com.twitter.scalding._ import serialization.Externalizer +import scala.annotation.meta.param + object BijectedSourceSink { type SourceSink[T] = TypedSource[T] with TypedSink[T] def apply[T, U](parent: SourceSink[T])(implicit transformer: ImplicitBijection[T, U]): BijectedSourceSink[T, U] = new BijectedSourceSink(parent)(transformer) } -class BijectedSourceSink[T, U](parent: BijectedSourceSink.SourceSink[T])(implicit @transient transformer: ImplicitBijection[T, U]) extends TypedSource[U] with TypedSink[U] { +class BijectedSourceSink[T, U](parent: BijectedSourceSink.SourceSink[T])(implicit @(transient @param) transformer: ImplicitBijection[T, U]) extends TypedSource[U] with TypedSink[U] { val lockedBij = Externalizer(transformer) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala index 78ed1159d2..fcc13444a3 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/CoGrouped.scala @@ -22,6 +22,7 @@ import cascading.pipe.{ CoGroup, Pipe } import com.twitter.scalding._ +import scala.annotation.meta.param import scala.collection.JavaConverters._ import com.twitter.scalding.serialization.Externalizer @@ -314,7 +315,7 @@ trait CoGrouped[K, +R] extends KeyedListLike[K, R, CoGrouped] with CoGroupable[K abstract class CoGroupedJoiner[K](inputSize: Int, getter: TupleGetter[K], - @transient inJoinFunction: (K, Iterator[CTuple], Seq[Iterable[CTuple]]) => Iterator[Any]) extends CJoiner { + @(transient @param) inJoinFunction: (K, Iterator[CTuple], Seq[Iterable[CTuple]]) => Iterator[Any]) extends CJoiner { /** * We have a test that should fail if Externalizer is not used here. @@ -360,7 +361,7 @@ abstract class CoGroupedJoiner[K](inputSize: Int, // If all the input pipes are unique, this works: class DistinctCoGroupJoiner[K](count: Int, getter: TupleGetter[K], - @transient joinF: (K, Iterator[CTuple], Seq[Iterable[CTuple]]) => Iterator[Any]) + @(transient @param) joinF: (K, Iterator[CTuple], Seq[Iterable[CTuple]]) => Iterator[Any]) extends CoGroupedJoiner[K](count, getter, joinF) { val distinctSize = count def distinctIndexOf(idx: Int) = idx diff --git a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala index 7dd8930ca2..73f272c502 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala @@ -31,6 +31,7 @@ import com.twitter.scalding.serialization.OrderedSerialization.Result import com.twitter.scalding.serialization.macros.impl.BinaryOrdering import com.twitter.scalding.serialization.macros.impl.BinaryOrdering._ +import scala.annotation.meta.param import scala.util.Try /** @@ -928,10 +929,10 @@ class TypedPipeFactory[T] private (@transient val next: NoStackAndThen[(FlowDef, /** * This is an instance of a TypedPipe that wraps a cascading Pipe */ -class TypedPipeInst[T] private[scalding] (@transient inpipe: Pipe, +class TypedPipeInst[T] private[scalding] (@(transient @param) inpipe: Pipe, fields: Fields, - @transient localFlowDef: FlowDef, - @transient val mode: Mode, + @(transient @param) localFlowDef: FlowDef, + @(transient @param) val mode: Mode, flatMapFn: FlatMapFn[T]) extends TypedPipe[T] { /** From 5301f1cef5dbfc6ab3ad192cc772f10b35015258 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Fri, 23 Oct 2015 17:46:17 +0200 Subject: [PATCH 15/60] =?UTF-8?q?Adjust=20to=20some=20Cascading=20API=20ch?= =?UTF-8?q?anges=20(getGraph=E2=86=92getElementGraph)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../scala/com/twitter/scalding/ExecutionContext.scala | 2 +- .../com/twitter/scalding/reducer_estimation/Common.scala | 8 +++++--- .../serialization/CascadingBinaryComparator.scala | 4 ++-- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala index 8c0497866a..5415454eb8 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala @@ -151,7 +151,7 @@ object ExecutionContext { private val LOG: Logger = LoggerFactory.getLogger(ExecutionContext.getClass) private[scalding] def getDesc[T](baseFlowStep: BaseFlowStep[T]): Seq[String] = { - baseFlowStep.getGraph.vertexSet.asScala.toSeq.flatMap(_ match { + baseFlowStep.getFlowNodeGraph.vertexSet.asScala.toSeq.flatMap(_ match { case pipe: Pipe => RichPipe.getPipeDescriptions(pipe) case _ => List() // no descriptions }) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index eb8ed048d4..c285c0deba 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -31,12 +31,14 @@ object Common { private def unrollTaps(taps: Seq[Tap[_, _, _]]): Seq[Tap[_, _, _]] = taps.flatMap { case multi: CompositeTap[_] => - unrollTaps(multi.getChildTaps.asScala.toSeq) + unrollTaps(multi.getChildTaps.asScala.map(x => x.asInstanceOf[Tap[_, _, _]]).toSeq) case t => Seq(t) } - def unrollTaps(step: FlowStep[_ <: JobConf]): Seq[Tap[_, _, _]] = - unrollTaps(step.getSources.asScala.toSeq) + def unrollTaps(step: FlowStep[_ <: JobConf]): Seq[Tap[_, _, _]] = { + val x = step.getFlowNodeGraph.getSourceTaps.asScala.toSeq + unrollTaps(x.toSeq) + } /** * Get the total size of the file(s) specified by the Hfs, which may contain a glob diff --git a/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala b/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala index 71777fad60..8d9acbe942 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/serialization/CascadingBinaryComparator.scala @@ -77,7 +77,7 @@ object CascadingBinaryComparator { def getDescriptionsForMissingOrdSer[U](bfs: BaseFlowStep[U]): Option[String] = // does this job have any Splices without OrderedSerialization: - if (bfs.getGraph.vertexSet.asScala.exists { + if (bfs.getElementGraph.vertexSet.asScala.exists { case gb: GroupBy => check(gb).isFailure case cg: CoGroup => check(cg).isFailure case _ => false // only do sorting in groupBy/cogroupBy @@ -94,7 +94,7 @@ object CascadingBinaryComparator { else { val badSteps = missing.size val msg = missing.zipWithIndex.map { case (msg, idx) => s"$msg" }.mkString - error(s"There are $badSteps missing OrderedSerializations: $msg") + sys.error(s"There are $badSteps missing OrderedSerializations: $msg") } } } From 96c29a192f043625b345ea3a41a57025ef86e696 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Fri, 23 Oct 2015 17:46:40 +0200 Subject: [PATCH 16/60] =?UTF-8?q?actually=20use=20the=20setMapSideAggregat?= =?UTF-8?q?ionThreshold=E2=86=92Capacity=20terminology=20change?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/scala/com/twitter/scalding/Config.scala | 4 ++-- scalding-core/src/main/scala/com/twitter/scalding/Job.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 4952bef9ea..9470851c53 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -420,7 +420,7 @@ object Config { empty .setListSpillThreshold(100 * 1000) .setMapSpillThreshold(100 * 1000) - .setMapSideAggregationThreshold(100 * 1000) + .setMapSideAggregationCapacity(100 * 1000) .setSerialization(Right(classOf[serialization.KryoHadoop])) .setScaldingVersion .setHRavenHistoryUserName @@ -520,7 +520,7 @@ object Config { (empty .setListSpillThreshold(100 * 1000) .setMapSpillThreshold(100 * 1000) - .setMapSideAggregationThreshold(100 * 1000) ++ fromHadoop(conf)) + .setMapSideAggregationCapacity(100 * 1000) ++ fromHadoop(conf)) .setSerialization(Right(classOf[serialization.KryoHadoop])) .setScaldingVersion /* diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala index 9fada4ac36..75f3fad069 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Job.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Job.scala @@ -172,7 +172,7 @@ class Job(val args: Args) extends FieldConversions with java.io.Serializable { val base = Config.empty .setListSpillThreshold(defaultSpillThreshold) .setMapSpillThreshold(defaultSpillThreshold) - .setMapSideAggregationThreshold(defaultSpillThreshold) + .setMapSideAggregationCapacity(defaultSpillThreshold) // This is setting a property for cascading/driven AppProps.addApplicationFramework(null, From 24ddd8c1b3e7d36926bb5501766608131bb6d752 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Fri, 23 Oct 2015 17:47:03 +0200 Subject: [PATCH 17/60] reference cascading3-enabled serialization libraries (all pull requests for now...) --- project/Build.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index f92dcff5fe..6270d6d9d6 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -24,9 +24,9 @@ object ScaldingBuild extends Build { val algebirdVersion = "0.11.0" val avroVersion = "1.7.4" val bijectionVersion = "0.8.1" - val cascadingAvroVersion = "2.1.2" + val cascadingAvroVersion = "3.0-SNAPSHOT" // https://github.com/ScaleUnlimited/cascading.avro/pull/44 val chillVersion = "0.7.1" - val elephantbirdVersion = "4.8" + val elephantbirdVersion = "4.11-SNAPSHOT" val hadoopLzoVersion = "0.4.19" val hadoopVersion = "2.5.0" val hbaseVersion = "0.94.10" @@ -324,7 +324,7 @@ object ScaldingBuild extends Build { "com.twitter" %% "bijection-core" % bijectionVersion, "com.twitter" %% "algebird-core" % algebirdVersion, "com.twitter" %% "chill" % chillVersion, - "com.twitter.elephantbird" % "elephant-bird-cascading2" % elephantbirdVersion, + "com.twitter.elephantbird" % "elephant-bird-cascading3" % elephantbirdVersion, "com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion, "com.hadoop.gplcompression" % "hadoop-lzo" % hadoopLzoVersion, // TODO: split this out into scalding-thrift @@ -350,7 +350,7 @@ object ScaldingBuild extends Build { lazy val scaldingParquet = module("parquet").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions - "org.apache.parquet" % "parquet-cascading" % parquetVersion + "org.apache.parquet" % "parquet-cascading3" % parquetVersion // FIXME: https://github.com/apache/parquet-mr/pull/284 exclude("org.apache.parquet", "parquet-pig") exclude("com.twitter.elephantbird", "elephant-bird-pig") exclude("com.twitter.elephantbird", "elephant-bird-core"), From b8723e333f6f7ca2cc504062a7c60f7003dee375 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Mon, 26 Oct 2015 09:33:47 +0100 Subject: [PATCH 18/60] =?UTF-8?q?(JobConf)=E2=86=92(=5F=20<:=20Configurati?= =?UTF-8?q?on)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../twitter/scalding/commons/source/LzoGenericScheme.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala index a81f97485e..449ef955c5 100644 --- a/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala +++ b/scalding-commons/src/main/scala/com/twitter/scalding/commons/source/LzoGenericScheme.scala @@ -124,8 +124,8 @@ class LzoGenericScheme[M](@(transient @param) conv: BinaryConverter[M], clazz: C new GenericWritable(convBox.get) override def sourceConfInit(fp: FlowProcess[_ <: Configuration], - tap: Tap[JobConf, RecordReader[_, _], OutputCollector[_, _]], - conf: JobConf): Unit = { + tap: Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]], + conf: Configuration): Unit = { LzoGenericScheme.setConverter(convBox.get, conf, SourceConfigBinaryConverterProvider.ProviderConfKey) MultiInputFormat.setClassConf(clazz, conf) From b08811d0b6300eee5d426040eae05b1c9a8139ba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Mon, 26 Oct 2015 10:28:29 +0100 Subject: [PATCH 19/60] Update some expected error messages coming from Cascading 3.0 --- .../src/test/scala/com/twitter/scalding/TypedFieldsTest.scala | 2 +- .../scalding/typed/RequireOrderedSerializationTest.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala index 65e0768335..b907d68316 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/TypedFieldsTest.scala @@ -27,7 +27,7 @@ class TypedFieldsTest extends WordSpec with Matchers { "throw an exception if a field is not comparable" in { val thrown = the[FlowException] thrownBy untypedJob - thrown.getMessage shouldBe "local step failed" + thrown.getMessage should startWith("local step failed") } // Now run the typed fields version diff --git a/scalding-core/src/test/scala/com/twitter/scalding/typed/RequireOrderedSerializationTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/typed/RequireOrderedSerializationTest.scala index 4d83c34cc2..1ce5ad73cb 100644 --- a/scalding-core/src/test/scala/com/twitter/scalding/typed/RequireOrderedSerializationTest.scala +++ b/scalding-core/src/test/scala/com/twitter/scalding/typed/RequireOrderedSerializationTest.scala @@ -55,7 +55,7 @@ class RequireOrderedSerializationTest extends WordSpec with Matchers { .run .finish } - ex.getMessage should include("SerializationTest.scala:29") + ex.getMessage should startWith("There are 1 missing OrderedSerializations:") } } "A OrderedSerJob" should { From d614c356c0e07ddc3fa32792a67f24bfe42b20fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Mon, 26 Oct 2015 10:29:02 +0100 Subject: [PATCH 20/60] Reinstate scala 2.10.5 cross-build --- project/Build.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 6270d6d9d6..0f6dfedc6e 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -50,9 +50,9 @@ object ScaldingBuild extends Build { val sharedSettings = Project.defaultSettings ++ assemblySettings ++ scalariformSettings ++ Seq( organization := "com.twitter", - scalaVersion := "2.11.7", + scalaVersion := "2.10.5", - crossScalaVersions := Seq(/* "2.10.5", */ "2.11.7"), + crossScalaVersions := Seq("2.10.5", "2.11.7"), ScalariformKeys.preferences := formattingPreferences, From fa3cfa85df87676d13d5df4fa63ad8da1a7c8506 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Mon, 26 Oct 2015 11:02:42 +0100 Subject: [PATCH 21/60] Record the Elephant-bird snapshot's pull request --- project/Build.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/Build.scala b/project/Build.scala index 0f6dfedc6e..0facfac76b 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -26,7 +26,7 @@ object ScaldingBuild extends Build { val bijectionVersion = "0.8.1" val cascadingAvroVersion = "3.0-SNAPSHOT" // https://github.com/ScaleUnlimited/cascading.avro/pull/44 val chillVersion = "0.7.1" - val elephantbirdVersion = "4.11-SNAPSHOT" + val elephantbirdVersion = "4.11-SNAPSHOT" // https://github.com/twitter/elephant-bird/pull/454 val hadoopLzoVersion = "0.4.19" val hadoopVersion = "2.5.0" val hbaseVersion = "0.94.10" From af482f696da4d80045536c9bf666e2b80b8a8b9e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Mon, 7 Dec 2015 12:55:14 +0100 Subject: [PATCH 22/60] Updating some upstream deps versions + exposing prebuilt dependencies' jars on conjars.org --- project/Build.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 0facfac76b..9bd8e62e34 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -24,7 +24,7 @@ object ScaldingBuild extends Build { val algebirdVersion = "0.11.0" val avroVersion = "1.7.4" val bijectionVersion = "0.8.1" - val cascadingAvroVersion = "3.0-SNAPSHOT" // https://github.com/ScaleUnlimited/cascading.avro/pull/44 + val cascadingAvroVersion = "3.0-SNAPSHOT" // https://github.com/ScaleUnlimited/cascading.avro/pull/44 val chillVersion = "0.7.1" val elephantbirdVersion = "4.11-SNAPSHOT" // https://github.com/twitter/elephant-bird/pull/454 val hadoopLzoVersion = "0.4.19" @@ -34,7 +34,7 @@ object ScaldingBuild extends Build { val jacksonVersion = "2.4.2" val json4SVersion = "3.2.11" val paradiseVersion = "2.0.1" - val parquetVersion = "1.8.1" + val parquetVersion = "1.8.2-SNAPSHOT" // https://github.com/apache/parquet-mr/pull/284 val protobufVersion = "2.4.1" val quasiquotesVersion = "2.0.1" val scalaCheckVersion = "1.12.2" @@ -76,7 +76,9 @@ object ScaldingBuild extends Build { "Concurrent Maven Repo" at "http://conjars.org/repo", "Clojars Repository" at "http://clojars.org/repo", "Twitter Maven" at "http://maven.twttr.com", - "Cloudera" at "https://repository.cloudera.com/artifactory/cloudera-repos/" + "Cloudera" at "https://repository.cloudera.com/artifactory/cloudera-repos/", + + "Conjars Repository" at "http://conjars.org/repo" /* TEMPORARY: to get at the depencencies' snapshots while preparing the PR1446 branch */ ), printDependencyClasspath := { From be20c6e369f3f32f52b4f8c5b5aa61e612e0b68d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 8 Dec 2015 10:23:28 +0100 Subject: [PATCH 23/60] Following advice from cwensel, depending on altered groupId/artifactId where dependencies are patched too (this is temporary) --- project/Build.scala | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 9bd8e62e34..b759ce3dc8 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -44,6 +44,9 @@ object ScaldingBuild extends Build { val slf4jVersion = "1.6.6" val thriftVersion = "0.5.0" val junitVersion = "4.10" + + /* NOTE: the temp.cchepelov.* groupIds are to let the scalding build machine access the patched upstream dependencies until they get merged. + This *must* be removed before proceeding. */ val printDependencyClasspath = taskKey[Unit]("Prints location of the dependencies") @@ -321,13 +324,14 @@ object ScaldingBuild extends Build { lazy val scaldingCommons = module("commons").settings( libraryDependencies ++= Seq( + "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", // TODO: split into scalding-protobuf "com.google.protobuf" % "protobuf-java" % protobufVersion, "com.twitter" %% "bijection-core" % bijectionVersion, "com.twitter" %% "algebird-core" % algebirdVersion, "com.twitter" %% "chill" % chillVersion, - "com.twitter.elephantbird" % "elephant-bird-cascading3" % elephantbirdVersion, - "com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion, + "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-cascading3" % elephantbirdVersion, + "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion, "com.hadoop.gplcompression" % "hadoop-lzo" % hadoopLzoVersion, // TODO: split this out into scalding-thrift "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", @@ -342,7 +346,7 @@ object ScaldingBuild extends Build { lazy val scaldingAvro = module("avro").settings( libraryDependencies ++= Seq( - "cascading.avro" % "avro-scheme" % cascadingAvroVersion, + "temp.cchepelov.cascading.avro" % "avro-scheme" % cascadingAvroVersion, "org.apache.avro" % "avro" % avroVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided" @@ -352,7 +356,10 @@ object ScaldingBuild extends Build { lazy val scaldingParquet = module("parquet").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions - "org.apache.parquet" % "parquet-cascading3" % parquetVersion // FIXME: https://github.com/apache/parquet-mr/pull/284 + "temp.cchepelov.org.apache.parquet" % "parquet-cascading3" % parquetVersion // FIXME: https://github.com/apache/parquet-mr/pull/284 + exclude("temp.cchepelov.org.apache.parquet", "parquet-pig") + exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-pig") + exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-core") exclude("org.apache.parquet", "parquet-pig") exclude("com.twitter.elephantbird", "elephant-bird-pig") exclude("com.twitter.elephantbird", "elephant-bird-core"), @@ -398,6 +405,9 @@ object ScaldingBuild extends Build { libraryDependencies ++= Seq( // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions "org.apache.parquet" % "parquet-cascading" % parquetVersion + exclude("temp.cchepelov.org.apache.parquet", "parquet-pig") + exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-pig") + exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-core") exclude("org.apache.parquet", "parquet-pig") exclude("com.twitter.elephantbird", "elephant-bird-pig") exclude("com.twitter.elephantbird", "elephant-bird-core"), @@ -485,7 +495,7 @@ object ScaldingBuild extends Build { "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "com.fasterxml.jackson.module" %% "jackson-module-scala" % jacksonVersion, "org.json4s" %% "json4s-native" % json4SVersion, - "com.twitter.elephantbird" % "elephant-bird-cascading2" % elephantbirdVersion % "provided" + "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-cascading3" % elephantbirdVersion % "provided" ) } ).dependsOn(scaldingCore) From b29be5a09024a2052f4b0a49b484833c7337ee10 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 8 Dec 2015 10:51:51 +0100 Subject: [PATCH 24/60] temporary groupId, yes, but correct temporary groupId --- project/Build.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/Build.scala b/project/Build.scala index b759ce3dc8..5a6b893498 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -26,7 +26,7 @@ object ScaldingBuild extends Build { val bijectionVersion = "0.8.1" val cascadingAvroVersion = "3.0-SNAPSHOT" // https://github.com/ScaleUnlimited/cascading.avro/pull/44 val chillVersion = "0.7.1" - val elephantbirdVersion = "4.11-SNAPSHOT" // https://github.com/twitter/elephant-bird/pull/454 + val elephantbirdVersion = "4.11-SNAPSHOT" // https://github.com/twitter/elephant-bird/pull/454 val hadoopLzoVersion = "0.4.19" val hadoopVersion = "2.5.0" val hbaseVersion = "0.94.10" From a35158938516b27bc97cfdc6e8c75108461bfb76 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 8 Dec 2015 13:00:31 +0100 Subject: [PATCH 25/60] (rollbacking the temp.cchepelov.* groupId namespace, causes too much busy work) + forgotten "cascading2" namespace in TypedJson --- project/Build.scala | 13 ++++++++----- .../main/scala/com/twitter/scalding/TypedJson.scala | 2 +- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 5a6b893498..2de37c4c56 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -356,8 +356,7 @@ object ScaldingBuild extends Build { lazy val scaldingParquet = module("parquet").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions - "temp.cchepelov.org.apache.parquet" % "parquet-cascading3" % parquetVersion // FIXME: https://github.com/apache/parquet-mr/pull/284 - exclude("temp.cchepelov.org.apache.parquet", "parquet-pig") + "org.apache.parquet" % "parquet-cascading3" % parquetVersion // FIXME: https://github.com/apache/parquet-mr/pull/284 exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-pig") exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-core") exclude("org.apache.parquet", "parquet-pig") @@ -404,15 +403,19 @@ object ScaldingBuild extends Build { .settings( libraryDependencies ++= Seq( // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions - "org.apache.parquet" % "parquet-cascading" % parquetVersion - exclude("temp.cchepelov.org.apache.parquet", "parquet-pig") + "org.apache.parquet" % "parquet-cascading3" % parquetVersion exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-pig") exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-core") exclude("org.apache.parquet", "parquet-pig") exclude("com.twitter.elephantbird", "elephant-bird-pig") exclude("com.twitter.elephantbird", "elephant-bird-core"), "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.apache.parquet" % "parquet-thrift" % parquetVersion % "test" classifier "tests", + "org.apache.parquet" % "parquet-thrift" % parquetVersion % "test" classifier "tests" + exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-pig") + exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-core") + exclude("org.apache.parquet", "parquet-pig") + exclude("com.twitter.elephantbird", "elephant-bird-pig") + exclude("com.twitter.elephantbird", "elephant-bird-core"), "com.twitter" %% "scrooge-serializer" % scroogeVersion, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "com.novocode" % "junit-interface" % "0.11" % "test", diff --git a/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala b/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala index 1dde3d1648..582b74237d 100644 --- a/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala +++ b/scalding-json/src/main/scala/com/twitter/scalding/TypedJson.scala @@ -2,7 +2,7 @@ package com.twitter.scalding import com.twitter.bijection.{ Injection, AbstractInjection } import com.twitter.bijection.Inversion._ -import com.twitter.elephantbird.cascading2.scheme.LzoTextLine +import com.twitter.elephantbird.cascading3.scheme.LzoTextLine import org.json4s._ import org.json4s.native.Serialization._ From c63ed497b735dfc9cfc89ec340dae0ab1e784b67 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 8 Dec 2015 13:11:20 +0100 Subject: [PATCH 26/60] Bumping source-compatibility to Java 8 was as bad an idea as ninja-dropping scalding 2.10. Oops. --- project/Build.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 2de37c4c56..2a2036fe31 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -59,9 +59,9 @@ object ScaldingBuild extends Build { ScalariformKeys.preferences := formattingPreferences, - javacOptions ++= Seq("-source", "1.8", "-target", "1.8"), + javacOptions ++= Seq("-source", "1.6", "-target", "1.6"), - javacOptions in doc := Seq("-source", "1.8"), + javacOptions in doc := Seq("-source", "1.6"), libraryDependencies ++= Seq( "org.mockito" % "mockito-all" % "1.8.5" % "test", From cd82ac8bd4f704ee08b176b70edc1070a0f32be5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 8 Dec 2015 13:37:53 +0100 Subject: [PATCH 27/60] (wip) add travis_wait calls to dodge 10-minute timeouts --- .travis.yml | 16 ++++++++-------- scripts/run_test.sh | 6 +++--- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/.travis.yml b/.travis.yml index a9a5a1ec11..5a94e5e012 100644 --- a/.travis.yml +++ b/.travis.yml @@ -19,7 +19,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-args scalding-date maple" + env: BUILD="base" TEST_TARGET="scalding-args scalding-date maple" TRAVIS_WAIT=travis_wait script: "scripts/run_test.sh" - scala: 2.10.6 @@ -27,7 +27,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons scalding-parquet scalding-parquet-scrooge" + env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons scalding-parquet scalding-parquet-scrooge" TRAVIS_WAIT=travis_wait script: "scripts/run_test.sh" - scala: 2.10.6 @@ -35,7 +35,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-core scalding-jdbc scalding-json scalding-db" + env: BUILD="base" TEST_TARGET="scalding-core scalding-jdbc scalding-json scalding-db" TRAVIS_WAIT=travis_wait script: "scripts/run_test.sh" - scala: 2.10.6 @@ -43,7 +43,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-hadoop-test" + env: BUILD="base" TEST_TARGET="scalding-hadoop-test" TRAVIS_WAIT=travis_wait script: "scripts/run_test.sh" - scala: 2.10.6 @@ -51,7 +51,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-serialization" + env: BUILD="base" TEST_TARGET="scalding-serialization" TRAVIS_WAIT=travis_wait script: "scripts/run_test.sh" - scala: 2.10.6 @@ -59,7 +59,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-thrift-macros" + env: BUILD="base" TEST_TARGET="scalding-thrift-macros" TRAVIS_WAIT=travis_wait script: "scripts/run_test.sh" - scala: 2.10.6 @@ -72,7 +72,7 @@ matrix: - "scripts/test_matrix_tutorials.sh" - scala: 2.11.7 - env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" + env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" TRAVIS_WAIT=travis_wait script: - "scripts/run_test.sh" - "scripts/build_assembly_no_test.sh scalding-assembly" @@ -91,7 +91,7 @@ matrix: - "scripts/test_execution_tutorial.sh" - scala: 2.11.7 - env: BUILD="test repl and typed tutorials" + env: BUILD="test repl and typed tutorials" TRAVIS_WAIT=travis_wait script: - "scripts/build_assembly_no_test.sh scalding-repl" - "scripts/test_repl_tutorial.sh" diff --git a/scripts/run_test.sh b/scripts/run_test.sh index 980f5c24c4..578fd9bd3e 100755 --- a/scripts/run_test.sh +++ b/scripts/run_test.sh @@ -19,17 +19,17 @@ PROGRESS_REPORTER_PID=$! echo "running..." -echo time ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd "test:compile")" +echo time $TRAVIS_WAIT ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd "test:compile")" export JVM_OPTS="-XX:+CMSClassUnloadingEnabled -XX:+UseConcMarkSweepGC -XX:ReservedCodeCacheSize=168m -XX:+TieredCompilation -XX:MaxPermSize=256m -Xms512m -Xmx1500m -Xss8m" -time ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd "test:compile")" # &> /dev/null +time $TRAVIS_WAIT ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd "test:compile")" # &> /dev/null kill -9 $PROGRESS_REPORTER_PID export JVM_OPTS="-XX:+CMSClassUnloadingEnabled -XX:+UseConcMarkSweepGC -XX:ReservedCodeCacheSize=128m -XX:+TieredCompilation -XX:MaxPermSize=256m -Xms256m -Xmx768m -Xss2m" echo "calling ... " echo "time ./sbt ++$TRAVIS_SCALA_VERSION $(withCmd test)" -time ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd test)" +time $TRAVIS_WAIT ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd test)" TST_EXIT_CODE=$? echo "all done" From 1dad07339434fe08409b28398cbaddc23cf703ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 8 Dec 2015 13:46:46 +0100 Subject: [PATCH 28/60] (wut? travis_wait is a bash function it seems) --- .travis.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.travis.yml b/.travis.yml index 5a94e5e012..5f6a3c3935 100644 --- a/.travis.yml +++ b/.travis.yml @@ -11,6 +11,7 @@ addons: apt: packages: - md5deep + - matrix: include: #BASE TESTS From 8dc8ebedd899c59515be179b2ee718f90e906622 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 8 Dec 2015 14:00:53 +0100 Subject: [PATCH 29/60] Revert "(wut? travis_wait is a bash function it seems)" This reverts commit 27f1d9aabbcb303bb75eaecbfde4485cf9eddd29. --- .travis.yml | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/.travis.yml b/.travis.yml index 5f6a3c3935..a9a5a1ec11 100644 --- a/.travis.yml +++ b/.travis.yml @@ -11,7 +11,6 @@ addons: apt: packages: - md5deep - - matrix: include: #BASE TESTS @@ -20,7 +19,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-args scalding-date maple" TRAVIS_WAIT=travis_wait + env: BUILD="base" TEST_TARGET="scalding-args scalding-date maple" script: "scripts/run_test.sh" - scala: 2.10.6 @@ -28,7 +27,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons scalding-parquet scalding-parquet-scrooge" TRAVIS_WAIT=travis_wait + env: BUILD="base" TEST_TARGET="scalding-avro scalding-hraven scalding-commons scalding-parquet scalding-parquet-scrooge" script: "scripts/run_test.sh" - scala: 2.10.6 @@ -36,7 +35,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-core scalding-jdbc scalding-json scalding-db" TRAVIS_WAIT=travis_wait + env: BUILD="base" TEST_TARGET="scalding-core scalding-jdbc scalding-json scalding-db" script: "scripts/run_test.sh" - scala: 2.10.6 @@ -44,7 +43,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-hadoop-test" TRAVIS_WAIT=travis_wait + env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" - scala: 2.10.6 @@ -52,7 +51,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-serialization" TRAVIS_WAIT=travis_wait + env: BUILD="base" TEST_TARGET="scalding-serialization" script: "scripts/run_test.sh" - scala: 2.10.6 @@ -60,7 +59,7 @@ matrix: script: "scripts/run_test.sh" - scala: 2.11.7 - env: BUILD="base" TEST_TARGET="scalding-thrift-macros" TRAVIS_WAIT=travis_wait + env: BUILD="base" TEST_TARGET="scalding-thrift-macros" script: "scripts/run_test.sh" - scala: 2.10.6 @@ -73,7 +72,7 @@ matrix: - "scripts/test_matrix_tutorials.sh" - scala: 2.11.7 - env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" TRAVIS_WAIT=travis_wait + env: BUILD="test tutorials and matrix tutorials and repl" TEST_TARGET="scalding-repl" script: - "scripts/run_test.sh" - "scripts/build_assembly_no_test.sh scalding-assembly" @@ -92,7 +91,7 @@ matrix: - "scripts/test_execution_tutorial.sh" - scala: 2.11.7 - env: BUILD="test repl and typed tutorials" TRAVIS_WAIT=travis_wait + env: BUILD="test repl and typed tutorials" script: - "scripts/build_assembly_no_test.sh scalding-repl" - "scripts/test_repl_tutorial.sh" From 9c4c25266c23f9e16a6d4daccf1f6bc7d0fcbf36 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 8 Dec 2015 14:01:04 +0100 Subject: [PATCH 30/60] Revert "(wip) add travis_wait calls to dodge 10-minute timeouts" This reverts commit 5bbdde7ef37c491c6e17bc4a3ac73c7f74e2f5bf. --- scripts/run_test.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/scripts/run_test.sh b/scripts/run_test.sh index 578fd9bd3e..980f5c24c4 100755 --- a/scripts/run_test.sh +++ b/scripts/run_test.sh @@ -19,17 +19,17 @@ PROGRESS_REPORTER_PID=$! echo "running..." -echo time $TRAVIS_WAIT ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd "test:compile")" +echo time ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd "test:compile")" export JVM_OPTS="-XX:+CMSClassUnloadingEnabled -XX:+UseConcMarkSweepGC -XX:ReservedCodeCacheSize=168m -XX:+TieredCompilation -XX:MaxPermSize=256m -Xms512m -Xmx1500m -Xss8m" -time $TRAVIS_WAIT ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd "test:compile")" # &> /dev/null +time ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd "test:compile")" # &> /dev/null kill -9 $PROGRESS_REPORTER_PID export JVM_OPTS="-XX:+CMSClassUnloadingEnabled -XX:+UseConcMarkSweepGC -XX:ReservedCodeCacheSize=128m -XX:+TieredCompilation -XX:MaxPermSize=256m -Xms256m -Xmx768m -Xss2m" echo "calling ... " echo "time ./sbt ++$TRAVIS_SCALA_VERSION $(withCmd test)" -time $TRAVIS_WAIT ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd test)" +time ./sbt -Dhttp.keepAlive=false -Dsbt.repository.secure=false ++$TRAVIS_SCALA_VERSION "$(withCmd test)" TST_EXIT_CODE=$? echo "all done" From 697b401ea1367ca8066005e0d2632e95fee130e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 8 Dec 2015 14:27:03 +0100 Subject: [PATCH 31/60] updating to latest released version of Cascading 3.0.x --- project/Build.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 2a2036fe31..d72e71ff1f 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -285,10 +285,10 @@ object ScaldingBuild extends Build { lazy val scaldingDate = module("date") lazy val cascadingVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.0.2") + System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.0.3") lazy val cascadingJDBCVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "3.0.0-wip-119") + System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "3.0.0-wip-120") lazy val scaldingBenchmarks = module("benchmarks").settings( libraryDependencies ++= Seq( From e05c248d0143861d689f50fdfb65a28d4737c3eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Thu, 7 Jan 2016 16:33:52 +0100 Subject: [PATCH 32/60] new InvalidSourceTap also needs Tap interface changes --- .../src/main/scala/com/twitter/scalding/FileSource.scala | 2 +- .../src/main/scala/com/twitter/scalding/Source.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala index cd369a3c94..5bfc3b6824 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/FileSource.scala @@ -288,7 +288,7 @@ abstract class FileSource extends SchemedSource with LocalSourceOverride with Hf } } - protected def createHdfsReadTap(hdfsMode: Hdfs): Tap[Configuration, _, _] = { + protected def createHdfsReadTap(hdfsMode: Hdfs): Tap[_ <: Configuration, _, _] = { val taps: List[Tap[Configuration, RecordReader[_, _], OutputCollector[_, _]]] = goodHdfsPaths(hdfsMode) .toList.map { path => CastHfsTap(createHfsTap(hdfsScheme, path, sinkMode)) } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Source.scala b/scalding-core/src/main/scala/com/twitter/scalding/Source.scala index 173099b609..7d68b325f4 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Source.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Source.scala @@ -60,7 +60,7 @@ class InvalidSourceTap(val hdfsPaths: Iterable[String]) extends SourceTap[JobCon override def getModifiedTime(conf: JobConf): Long = 0L - override def openForRead(flow: FlowProcess[JobConf], input: RecordReader[_, _]): TupleEntryIterator = + override def openForRead(flow: FlowProcess[_ <: JobConf], input: RecordReader[_, _]): TupleEntryIterator = sys.error(s"InvalidSourceTap: No good paths in $hdfsPaths") override def resourceExists(conf: JobConf): Boolean = false @@ -76,7 +76,7 @@ class InvalidSourceTap(val hdfsPaths: Iterable[String]) extends SourceTap[JobCon // 4. source.validateTaps (throws InvalidSourceException) // In the worst case if the flow plan is misconfigured, // openForRead on mappers should fail when using this tap. - override def sourceConfInit(flow: FlowProcess[JobConf], conf: JobConf): Unit = { + override def sourceConfInit(flow: FlowProcess[_ <: JobConf], conf: JobConf): Unit = { conf.setInputFormat(classOf[cascading.tap.hadoop.io.MultiInputFormat]) super.sourceConfInit(flow, conf) } From 45ade969f67e87819f570e57ee43e5c7b81e25a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 13 Jan 2016 16:54:04 +0100 Subject: [PATCH 33/60] new: ConfigBridge, in order to deal with the various [Config] types FlowSteps et al. can carry (before Cascading 3.0, this was a JobConf on -hadoop, Properties on -local) --- .../scala/com/twitter/scalding/Config.scala | 4 +- .../com/twitter/scalding/ConfigBridge.scala | 46 +++++++++++++++++++ 2 files changed, 47 insertions(+), 3 deletions(-) create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 9470851c53..458cc6a59b 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -507,9 +507,7 @@ object Config { * This copy also forces all expressions in values to be evaluated, freezing them * as well. */ - def fromHadoop(conf: Configuration): Config = - // use `conf.get` to force JobConf to evaluate expressions - Config(conf.asScala.map { e => e.getKey -> conf.get(e.getKey) }.toMap) + def fromHadoop(anyConf: Any): Config = ConfigBridge.fromPlatform(anyConf) /* * For everything BUT SERIALIZATION, this prefers values in conf, diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala b/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala new file mode 100644 index 0000000000..3934c7a620 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala @@ -0,0 +1,46 @@ +package com.twitter.scalding + +import cascading.flow.FlowStep + +import scala.collection.JavaConverters._ + +/** + * Created by cchepelov on 13/01/16. + */ +object ConfigBridge { + /** + * This adapter handles the various vessel types that can be used to configure properties of a a + * FlowStep[_] since Cascading 3.0 + */ + class FlowStepAdapter(flowStep: FlowStep[_]) { + def getConfigValue(key: String): String = + flowStep.getConfig match { + case conf: org.apache.hadoop.conf.Configuration => conf.get(key) + case conf: org.apache.commons.configuration.Configuration => conf.getString(key) + case conf: java.util.Properties => conf.getProperty(key) + case _ => throw new NotImplementedError(s"unknown flowStep Config type ${flowStep.getConfig.getClass}") + } + + def setConfigValue(name: String, value: String): FlowStep[_] = { + flowStep.getConfig match { + case conf: org.apache.hadoop.conf.Configuration => conf.set(name, value) + case conf: org.apache.commons.configuration.Configuration => conf.addProperty(name, value) + case conf: java.util.Properties => conf.put(name, value) + case _ => throw new NotImplementedError(s"unknown flowStep Config type ${flowStep.getConfig.getClass}") + } + flowStep + } + } + + def fromPlatform(anyConf: Any): Config = { + anyConf match { + // use `conf.get` to force JobConf to evaluate expressions + case conf: org.apache.hadoop.conf.Configuration => Config(conf.asScala.map { e => e.getKey -> conf.get(e.getKey) }.toMap) + case conf: org.apache.commons.configuration.Configuration => Config(conf.getKeys.asScala.map { k => k.asInstanceOf[String] }.map { (k: String) => k -> conf.getString(k) }.toMap) + case conf: java.util.Properties => Config(conf.asScala.map { e => e._1 -> conf.getProperty(e._1) }.toMap) + case _ => throw new NotImplementedError(s"Can't get from 'hadoop' with configuration type ${anyConf.getClass}") + } + } + + implicit def toFlowStepPimpFromFlowStep(flowStep: FlowStep[_]): FlowStepAdapter = new FlowStepAdapter(flowStep) +} \ No newline at end of file From 835cf37f50b287dcbbb9e203fa2f5385f8ff4182 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 13 Jan 2016 16:55:43 +0100 Subject: [PATCH 34/60] =?UTF-8?q?cascading=203.0:=20Flow[JobConf]=20?= =?UTF-8?q?=E2=86=92=20Flow[=5F]=20where=20=5F=20can=20be=20JobConf,=20Con?= =?UTF-8?q?figuration,=20Property...?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../scalding/platform/HadoopPlatformJobTest.scala | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala index 5075544915..45f1eb0f6a 100644 --- a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/HadoopPlatformJobTest.scala @@ -15,6 +15,7 @@ limitations under the License. */ package com.twitter.scalding.platform +import scala.collection.JavaConverters._ import cascading.flow.Flow import com.twitter.scalding._ import com.twitter.scalding.source.TypedText @@ -37,7 +38,7 @@ case class HadoopPlatformJobTest( dataToCreate: Seq[(String, Seq[String])] = Vector(), sourceWriters: Seq[Args => Job] = Vector.empty, sourceReaders: Seq[Mode => Unit] = Vector.empty, - flowCheckers: Seq[Flow[JobConf] => Unit] = Vector.empty) { + flowCheckers: Seq[Flow[_] => Unit] = Vector.empty) { private val LOG = LoggerFactory.getLogger(getClass) def arg(inArg: String, value: List[String]): HadoopPlatformJobTest = copy(argsMap = argsMap + (inArg -> value)) @@ -59,7 +60,7 @@ case class HadoopPlatformJobTest( def sink[T](in: Mappable[T])(toExpect: Seq[T] => Unit): HadoopPlatformJobTest = copy(sourceReaders = sourceReaders :+ { m: Mode => toExpect(in.toIterator(Config.defaultFrom(m), m).toSeq) }) - def inspectCompletedFlow(checker: Flow[JobConf] => Unit): HadoopPlatformJobTest = + def inspectCompletedFlow(checker: Flow[_] => Unit): HadoopPlatformJobTest = copy(flowCheckers = flowCheckers :+ checker) private def createSources() { @@ -98,7 +99,7 @@ case class HadoopPlatformJobTest( checkSinks() flowCheckers.foreach { checker => job.completedFlow.collect { - case f: Flow[JobConf] => checker(f) + case f: Flow[_] => checker(f) } } } @@ -107,6 +108,13 @@ case class HadoopPlatformJobTest( @annotation.tailrec private final def runJob(job: Job) { + // create cascading 3.0 planner trace files during tests + if (System.getenv.asScala.getOrElse("SCALDING_CASCADING3_DEBUG", "0") == "1") { + System.setProperty("cascading.planner.plan.path", "target/test/cascading/traceplan/" + job.name) + System.setProperty("cascading.planner.plan.transforms.path", "target/test/cascading/traceplan/" + job.name + "/transform") + System.setProperty("cascading.planner.stats.path", "target/test/cascading/traceplan/" + job.name + "/stats") + } + job.run job.clear job.next match { From ef284409c805ab094f93c8f04141a501afa0957b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 13 Jan 2016 16:57:00 +0100 Subject: [PATCH 35/60] Adjust how Pipe description texts are carried + update tests --- .../twitter/scalding/ExecutionContext.scala | 25 +++++----- .../scalding/platform/PlatformTest.scala | 49 ++++++++++--------- 2 files changed, 40 insertions(+), 34 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala index 5415454eb8..8b185ba31c 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala @@ -15,9 +15,11 @@ limitations under the License. */ package com.twitter.scalding +import java.util.Properties + import cascading.flow.hadoop.HadoopFlow -import cascading.flow.{ Flow, FlowDef, FlowListener, FlowStepListener, FlowStepStrategy } -import cascading.flow.planner.BaseFlowStep +import cascading.flow._ +import cascading.flow.planner.{ BaseFlowNode, BaseFlowStep } import cascading.pipe.Pipe import com.twitter.scalding.reducer_estimation.ReducerEstimatorStepStrategy import com.twitter.scalding.serialization.CascadingBinaryComparator @@ -43,11 +45,11 @@ trait ExecutionContext { if (descriptions.nonEmpty) Some(descriptions.distinct.mkString(", ")) else None } - private def updateStepConfigWithDescriptions(step: BaseFlowStep[JobConf]): Unit = { - val conf = step.getConfig - getIdentifierOpt(ExecutionContext.getDesc(step)).foreach(descriptionString => { - conf.set(Config.StepDescriptions, descriptionString) - }) + private def updateStepConfigWithDescriptions(step: BaseFlowStep[_]): Unit = { + import ConfigBridge._ + + getIdentifierOpt(ExecutionContext.getDesc(step)) + .foreach(descriptionString => step.setConfigValue(Config.StepDescriptions, descriptionString)) } final def buildFlow: Try[Flow[_]] = @@ -79,11 +81,12 @@ trait ExecutionContext { } flow match { - case hadoopFlow: HadoopFlow => - val flowSteps = hadoopFlow.getFlowSteps.asScala + case baseFlow: BaseFlow[_] => + val flowSteps = baseFlow.getFlowSteps.asScala flowSteps.foreach { - case baseFlowStep: BaseFlowStep[JobConf] => + case baseFlowStep: BaseFlowStep[_] => updateStepConfigWithDescriptions(baseFlowStep) + case anyOtherBaseFlowStep => throw new NotImplementedError("unknown flowStep type ${anyOtherBaseFlowStep.getClass}") } case _ => // descriptions not yet supported in other modes } @@ -151,7 +154,7 @@ object ExecutionContext { private val LOG: Logger = LoggerFactory.getLogger(ExecutionContext.getClass) private[scalding] def getDesc[T](baseFlowStep: BaseFlowStep[T]): Seq[String] = { - baseFlowStep.getFlowNodeGraph.vertexSet.asScala.toSeq.flatMap(_ match { + baseFlowStep.getElementGraph.vertexSet.asScala.toSeq.flatMap(_ match { case pipe: Pipe => RichPipe.getPipeDescriptions(pipe) case _ => List() // no descriptions }) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala index c8f27752c6..e0345778ce 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala @@ -15,6 +15,8 @@ limitations under the License. */ package com.twitter.scalding.platform +import cascading.flow.FlowStep +import cascading.flow.planner.BaseFlowStep import cascading.pipe.joiner.{ JoinerClosure, InnerJoin } import cascading.tuple.Tuple @@ -271,6 +273,7 @@ class CheckForFlowProcessInTypedJob(args: Args) extends Job(args) { // Keeping all of the specifications in the same tests puts the result output all together at the end. // This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest { + import ConfigBridge._ "An InAndOutTest" should { val inAndOut = Seq("a", "b", "c") @@ -321,15 +324,16 @@ class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest "A TypedPipeForceToDiskWithDescriptionPipe" should { "have a custom step name from withDescription" in { + HadoopPlatformJobTest(new TypedPipeForceToDiskWithDescriptionJob(_), cluster) .inspectCompletedFlow { flow => val steps = flow.getFlowSteps.asScala - val firstStep = steps.filter(_.getName.startsWith("(1/2")) - val secondStep = steps.filter(_.getName.startsWith("(2/2")) - val lab1 = firstStep.map(_.getConfig.get(Config.StepDescriptions)) + val firstStep = steps.filter(_.getName.startsWith("(1/2)")) + val secondStep = steps.filter(_.getName.startsWith("(2/2)")) + val lab1 = firstStep.map(_.getConfigValue(Config.StepDescriptions)) lab1 should have size 1 lab1(0) should include ("write words to disk") - val lab2 = secondStep.map(_.getConfig.get(Config.StepDescriptions)) + val lab2 = secondStep.map(_.getConfigValue(Config.StepDescriptions)) lab2 should have size 1 lab2(0) should include ("output frequency by length") } @@ -343,14 +347,15 @@ class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest .inspectCompletedFlow { flow => val steps = flow.getFlowSteps.asScala steps should have size 1 - val firstStep = steps.headOption.map(_.getConfig.get(Config.StepDescriptions)).getOrElse("") - val lines = List(147, 150, 154).map { i => - s"com.twitter.scalding.platform.TypedPipeJoinWithDescriptionJob.(PlatformTest.scala:$i" - } - firstStep should include ("leftJoin") - firstStep should include ("hashJoin") - lines.foreach { l => firstStep should include (l) } - steps.map(_.getConfig.get(Config.StepDescriptions)).foreach(s => info(s)) + val firstStepDescs = steps.headOption.map(_.getConfigValue(Config.StepDescriptions)).getOrElse("") + val firstStepDescSet = firstStepDescs.split(",").map(_.trim).toSet + + val expected = Set(149, 151, 152, 155, 156).map { i => + s"com.twitter.scalding.platform.TypedPipeJoinWithDescriptionJob.(PlatformTest.scala:$i)" + } ++ Seq("leftJoin", "hashJoin") + + firstStepDescSet should equal(expected) + steps.map(_.getConfigValue(Config.StepDescriptions)).foreach(s => info(s)) } .run } @@ -361,18 +366,16 @@ class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest HadoopPlatformJobTest(new TypedPipeWithDescriptionJob(_), cluster) .inspectCompletedFlow { flow => val steps = flow.getFlowSteps.asScala - val descs = List("map stage - assign words to 1", + val expectedDescs = Set("map stage - assign words to 1", "reduce stage - sum", - "write", - // should see the .group and the .write show up as line numbers - "com.twitter.scalding.platform.TypedPipeWithDescriptionJob.(PlatformTest.scala:137)", - "com.twitter.scalding.platform.TypedPipeWithDescriptionJob.(PlatformTest.scala:141)") - - val foundDescs = steps.map(_.getConfig.get(Config.StepDescriptions)) - descs.foreach { d => - assert(foundDescs.size == 1) - assert(foundDescs(0).contains(d)) - } + "write") ++ + Seq(138, 139, 141, 142, 143).map( + linenum => s"com.twitter.scalding.platform.TypedPipeWithDescriptionJob.(PlatformTest.scala:${linenum})") + + val foundDescs = steps.map(_.getConfigValue(Config.StepDescriptions).split(",").map(_.trim).toSet) + foundDescs should have size 1 + + foundDescs.head should equal(expectedDescs) //steps.map(_.getConfig.get(Config.StepDescriptions)).foreach(s => info(s)) } .run From 6f0f5eea4f672dda7f3ebad982f8b13637434bab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 13 Jan 2016 16:57:54 +0100 Subject: [PATCH 36/60] Adjust how the numReducers property is accessed (cascading 3.0) --- .../RatioBasedEstimatorTest.scala | 16 ++++++------- .../ReducerEstimatorTest.scala | 2 +- .../RuntimeReducerEstimatorTest.scala | 24 +++++++++---------- 3 files changed, 21 insertions(+), 21 deletions(-) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala index c467a0de7c..496c9ff0b9 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala @@ -128,8 +128,8 @@ class RatioBasedReducerEstimatorTest extends WordSpec with Matchers with HadoopS val steps = flow.getFlowSteps.asScala steps should have size 1 - val conf = steps.head.getConfig - conf.getNumReduceTasks should equal (1) // default + val conf = Config.fromHadoop(steps.head.getConfig) + conf.getNumReducers should equal (1) // default } .run } @@ -144,8 +144,8 @@ class RatioBasedReducerEstimatorTest extends WordSpec with Matchers with HadoopS val steps = flow.getFlowSteps.asScala steps should have size 1 - val conf = steps.head.getConfig - conf.getNumReduceTasks should equal (1) // default + val conf = Config.fromHadoop(steps.head.getConfig) + conf.getNumReducers should equal (1) // default } .run } @@ -164,8 +164,8 @@ class RatioBasedReducerEstimatorTest extends WordSpec with Matchers with HadoopS // base estimate from input size reducer = 3 // reducer ratio from history = 0.5 // final estimate = ceil(3 * 0.5) = 2 - val conf = steps.head.getConfig - conf.getNumReduceTasks should equal (2) + val conf = Config.fromHadoop(steps.head.getConfig) + conf.getNumReducers should equal (2) } .run } @@ -180,8 +180,8 @@ class RatioBasedReducerEstimatorTest extends WordSpec with Matchers with HadoopS val steps = flow.getFlowSteps.asScala steps should have size 1 - val conf = steps.head.getConfig - conf.getNumReduceTasks should equal (1) // default + val conf = Config.fromHadoop(steps.head.getConfig) + conf.getNumReducers should equal (1) // default } .run } diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index 35f7bbf85d..b0e2974ce2 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -146,7 +146,7 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf .sink[Double](out)(_.head shouldBe 2.86 +- 0.0001) .inspectCompletedFlow { flow => val steps = flow.getFlowSteps.asScala - val reducers = steps.map(_.getConfig.getInt(Config.HadoopNumReducers, 0)).toList + val reducers = steps.map(step => Config.fromHadoop(step.getConfig).getNumReducers.getOrElse(0)).toList reducers shouldBe List(3, 1, 1) } .run diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala index 71f460c26a..5fe85494cd 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala @@ -49,7 +49,7 @@ class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopShar val steps = flow.getFlowSteps.asScala assert(steps.length == 1) - val conf = steps.head.getConfig + val conf = Config.fromHadoop(steps.head.getConfig) // So our histories are (taking median runtimes): // // 2 * inputSize bytes, 3 reducers * 1000 ms for each reducer @@ -66,7 +66,7 @@ class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopShar // so we anticipate that processing (inputSize bytes) // will take 1500 ms total. // To do this in 25 ms, we need 60 reducers. - assert(conf.getNumReduceTasks == 60) + assert(conf.getNumReducers.get == 60) } .run } @@ -83,7 +83,7 @@ class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopShar val steps = flow.getFlowSteps.asScala assert(steps.length == 1) - val conf = steps.head.getConfig + val conf = Config.fromHadoop(steps.head.getConfig) // So our histories are (taking mean runtimes): // // 2 * inputSize bytes, 3 reducers * 1336.67 ms for each reducer @@ -101,7 +101,7 @@ class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopShar // will take 1525.8 ms total. // // To do this in 25 ms, we need 61.03 reducers, which rounds up to 62. - assert(conf.getNumReduceTasks == 62) + assert(conf.getNumReducers.get == 62) } .run } @@ -118,7 +118,7 @@ class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopShar val steps = flow.getFlowSteps.asScala assert(steps.length == 1) - val conf = steps.head.getConfig + val conf = Config.fromHadoop(steps.head.getConfig) // So our histories are (taking mean runtimes): // // 2 * inputSize bytes, 3 reducers * 1337 ms for each reducer @@ -131,7 +131,7 @@ class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopShar // that the job will take 3342 ms total. // // To do this in 25 ms, we need 134 reducers. - assert(conf.getNumReduceTasks == 134) + assert(conf.getNumReducers.get == 134) } .run } @@ -148,7 +148,7 @@ class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopShar val steps = flow.getFlowSteps.asScala assert(steps.length == 1) - val conf = steps.head.getConfig + val conf = Config.fromHadoop(steps.head.getConfig) // So our histories are (taking median runtimes): // // 2 * inputSize bytes, 3 reducers * 1000 ms for each reducer @@ -161,7 +161,7 @@ class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopShar // that the job will take 3000 ms total. // // To do this in 25 ms, we need 120 reducers. - assert(conf.getNumReduceTasks == 120) + assert(conf.getNumReducers.get == 120) } .run } @@ -176,12 +176,12 @@ class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopShar val steps = flow.getFlowSteps.asScala assert(steps.length == 1) - val conf = steps.head.getConfig + val conf = Config.fromHadoop(steps.head.getConfig) // EmptyRuntimeEstimator should have returned None, // so it should have fallen back to DummyEstimator, // which returns 42. - assert(conf.getNumReduceTasks == 42) + assert(conf.getNumReducers.get == 42) } } @@ -195,12 +195,12 @@ class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopShar val steps = flow.getFlowSteps.asScala assert(steps.length == 1) - val conf = steps.head.getConfig + val conf = Config.fromHadoop(steps.head.getConfig) // ErrorRuntimeEstimator should have returned None, // so it should have fallen back to DummyEstimator, // which returns 42. - assert(conf.getNumReduceTasks == 42) + assert(conf.getNumReducers.get == 42) } } } From caf4f5f6338aa8b3aa7843d07a477c080557207c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 13 Jan 2016 16:58:11 +0100 Subject: [PATCH 37/60] cascading 3.0 flow step id is now alphanumeric, not numeric --- .../hraven/reducer_estimation/HRavenHistoryService.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala index 21d7cf3bfa..2e5b9a58a5 100644 --- a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala +++ b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala @@ -125,7 +125,7 @@ object HRavenHistoryService extends HistoryService { def findMatchingJobStep(pastFlow: Flow) = pastFlow.getJobs.asScala.find { step => try { - step.getConfiguration.get("cascading.flow.step.id").toInt == stepId + step.getConfiguration.get("cascading.flow.step.id") == stepId } catch { case _: NumberFormatException => false } From 8e850ecddf1bf107292785e824cdeebf47c81d51 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 13 Jan 2016 16:59:04 +0100 Subject: [PATCH 38/60] update to currently locally-built cascading WIP. FIXME ASAP: use WIP post https://github.com/cwensel/cascading/pull/47 --- .gitignore | 2 ++ project/Build.scala | 12 ++++++++++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 8421ffc531..0182e6b9aa 100644 --- a/.gitignore +++ b/.gitignore @@ -34,3 +34,5 @@ tutorial/data/avrooutput0.avro .scalding_repl scalding-hadoop-test/NOTICE NOTICE +*~ +build/ \ No newline at end of file diff --git a/project/Build.scala b/project/Build.scala index d72e71ff1f..4c3560a776 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -285,7 +285,7 @@ object ScaldingBuild extends Build { lazy val scaldingDate = module("date") lazy val cascadingVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.0.3") + System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.0.4-wip-dev") // TEMPORARY. This is how a local build ends up named on my machine today -- cchepelov, 2016-01-13 lazy val cascadingJDBCVersion = System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "3.0.0-wip-120") @@ -302,7 +302,7 @@ object ScaldingBuild extends Build { lazy val scaldingCore = module("core").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "cascading" % "cascading-core" % cascadingVersion, - "cascading" % "cascading-hadoop" % cascadingVersion, + "cascading" % "cascading-hadoop" % cascadingVersion, // ought to become "provided" or no longer here. "cascading" % "cascading-local" % cascadingVersion, "com.twitter" % "chill-hadoop" % chillVersion, "com.twitter" % "chill-java" % chillVersion, @@ -417,6 +417,8 @@ object ScaldingBuild extends Build { exclude("com.twitter.elephantbird", "elephant-bird-pig") exclude("com.twitter.elephantbird", "elephant-bird-core"), "com.twitter" %% "scrooge-serializer" % scroogeVersion, + "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion, + "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-pig" % elephantbirdVersion, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "com.novocode" % "junit-interface" % "0.11" % "test", "junit" % "junit" % junitVersion % "test" @@ -521,6 +523,7 @@ object ScaldingBuild extends Build { "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion classifier "tests", "org.apache.hadoop" % "hadoop-common" % hadoopVersion classifier "tests", "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % hadoopVersion classifier "tests", + // ought to depend here on "cascading" % "cascading-hadoop" % cascadingVersion "com.twitter" %% "chill-algebird" % chillVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, @@ -530,6 +533,11 @@ object ScaldingBuild extends Build { } ).dependsOn(scaldingCore, scaldingSerialization) + /* cchepelov 2016-01-13 Question: create scaldingHadoop2MR1Test, scaldingHadoop2TezTest modules here? Depending on a scalding-hadoop-common ? + * scaldingHadoopTest & scaldingHadoop2MR1Test can probably share the reducer_estimation tests. Much less sure about Tez. + * */ + + // This one uses a different naming convention lazy val maple = Project( id = "maple", From 537240ee471d88ea2ee569274ec74890e23066ed Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Thu, 14 Jan 2016 10:37:05 +0100 Subject: [PATCH 39/60] Cheat a bit on the dfs-datastores VersionedKeyValSourceTest dfs-datastores' semantics have shifted since 1.3.4, and now it is of the consumer's responsibility to create the empty version directory which will contain the version's data. In this whitebox test, nobody was creating the empty directory anymore, so the test failed. Added a "hint" and an explanation (not sure this tests much now, but keeps the original dance mostly in) --- .../scalding/commons/VersionedKeyValSourceTest.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/scalding-commons/src/test/scala/com/twitter/scalding/commons/VersionedKeyValSourceTest.scala b/scalding-commons/src/test/scala/com/twitter/scalding/commons/VersionedKeyValSourceTest.scala index 74b0dd4c67..8d01cc78c7 100644 --- a/scalding-commons/src/test/scala/com/twitter/scalding/commons/VersionedKeyValSourceTest.scala +++ b/scalding-commons/src/test/scala/com/twitter/scalding/commons/VersionedKeyValSourceTest.scala @@ -15,6 +15,7 @@ limitations under the License. */ package com.twitter.scalding.commons.source +import org.apache.hadoop.fs.Path import org.scalatest.{ Matchers, WordSpec } import com.twitter.scalding._ import com.twitter.scalding.commons.datastores.VersionedStore; @@ -135,7 +136,12 @@ class VersionedKeyValSourceTest extends WordSpec with Matchers { val store = new VersionedStore(root.getAbsolutePath) versions foreach { v => val p = store.createVersion(v) + /* since dfs-datastores 1.3.5, store.succeedVersion() will fail if the directory + doesn't exist, and it won't exist after createVersion() until data is actually inserted. + So we cheat here to keep the test mechanism alive + */ new File(p).mkdirs() + /* /cheat */ store.succeedVersion(p) } From 408f2e83165f579afe78ea47a27a7535a448bbf7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Thu, 14 Jan 2016 17:03:05 +0100 Subject: [PATCH 40/60] Reductor estimate test failed, as Hadoop2.6 understands the legacy properties but responds in the current --- .../main/scala/com/twitter/scalding/Config.scala | 15 ++++++++++++--- .../scala/com/twitter/scalding/ConfigBridge.scala | 5 +++++ .../scalding/reducer_estimation/Common.scala | 4 +++- .../RatioBasedEstimatorTest.scala | 14 ++++++++++---- 4 files changed, 30 insertions(+), 8 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 458cc6a59b..fdde19ee03 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -360,8 +360,11 @@ trait Config extends Serializable { .toList /** Get the number of reducers (this is the parameter Hadoop will use) */ - def getNumReducers: Option[Int] = get(Config.HadoopNumReducers).map(_.toInt) - def setNumReducers(n: Int): Config = this + (Config.HadoopNumReducers -> n.toString) + def getNumReducers: Option[Int] = get(Config.HadoopNumReducersLegacy) + .orElse(get(Config.HadoopNumReducers2)) + .map(_.toInt) + + def setNumReducers(n: Int): Config = this + (Config.HadoopNumReducersLegacy -> n.toString) // Note: setting the legacy key for cascading-hadoop compat, hadoop-2.6.0 still accepts it. /** Set username from System.used for querying hRaven. */ def setHRavenHistoryUserName: Config = @@ -395,7 +398,13 @@ object Config { * Parameter that actually controls the number of reduce tasks. * Be sure to set this in the JobConf for the *step* not the flow. */ - val HadoopNumReducers = "mapred.reduce.tasks" + val HadoopNumReducersLegacy = "mapred.reduce.tasks" + val HadoopNumReducers2 = "mapreduce.job.reduces" + + @deprecated( + message = "please select between HadoopNumReducersLegacy or HadoopNumReducers2. Or use getNumReducers()", + since = "2016-01-16") + val HadoopNumReducers = HadoopNumReducersLegacy // kept for source-level compatibility, for now (RFC)d /** Name of parameter to specify which class to use as the default estimator. */ val ReducerEstimators = "scalding.reducer.estimator.classes" diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala b/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala index 3934c7a620..ef3fa86001 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala @@ -34,6 +34,11 @@ object ConfigBridge { def fromPlatform(anyConf: Any): Config = { anyConf match { + /* NOTE: for now we always return a Config instance (actually, an anonymous realization of the Config trait) + no matter what the underlying fabric/platform. Here would be a GREAT opportunity to return a specific Config + implementation (notably, to deal with things like HadoopNumReducers* ) + */ + // use `conf.get` to force JobConf to evaluate expressions case conf: org.apache.hadoop.conf.Configuration => Config(conf.asScala.map { e => e.getKey -> conf.get(e.getKey) }.toMap) case conf: org.apache.commons.configuration.Configuration => Config(conf.getKeys.asScala.map { k => k.asInstanceOf[String] }.map { (k: String) => k -> conf.getString(k) }.toMap) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala index c285c0deba..d3d330be3e 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala @@ -158,7 +158,9 @@ object ReducerEstimatorStepStrategy extends FlowStepStrategy[JobConf] { preds: JList[FlowStep[JobConf]], step: FlowStep[JobConf]): Unit = { val conf = step.getConfig - val stepNumReducers = conf.get(Config.HadoopNumReducers) + val stepNumReducers = Option(conf.get(Config.HadoopNumReducersLegacy)) + .orElse(Option(conf.get(Config.HadoopNumReducers2))) + .getOrElse("-1") // whether the reducers have been set explicitly with `withReducers` val setExplicitly = conf.getBoolean(Config.WithReducersSetExplicitly, false) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala index 496c9ff0b9..7c9a11d806 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala @@ -129,7 +129,9 @@ class RatioBasedReducerEstimatorTest extends WordSpec with Matchers with HadoopS steps should have size 1 val conf = Config.fromHadoop(steps.head.getConfig) - conf.getNumReducers should equal (1) // default + + val numReducers = conf.getNumReducers + conf.getNumReducers.get should equal (1) // default } .run } @@ -145,7 +147,7 @@ class RatioBasedReducerEstimatorTest extends WordSpec with Matchers with HadoopS steps should have size 1 val conf = Config.fromHadoop(steps.head.getConfig) - conf.getNumReducers should equal (1) // default + conf.getNumReducers.get should equal (1) // default } .run } @@ -165,7 +167,9 @@ class RatioBasedReducerEstimatorTest extends WordSpec with Matchers with HadoopS // reducer ratio from history = 0.5 // final estimate = ceil(3 * 0.5) = 2 val conf = Config.fromHadoop(steps.head.getConfig) - conf.getNumReducers should equal (2) + + val numReducers = conf.getNumReducers + conf.getNumReducers.get should equal (2) } .run } @@ -181,7 +185,9 @@ class RatioBasedReducerEstimatorTest extends WordSpec with Matchers with HadoopS steps should have size 1 val conf = Config.fromHadoop(steps.head.getConfig) - conf.getNumReducers should equal (1) // default + + val numReducers = conf.getNumReducers + conf.getNumReducers.get should equal (1) // default } .run } From 68356d6ca61cdb2406fb2bdf36a16564fa85bf64 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov?= Date: Fri, 15 Jan 2016 14:35:40 +0100 Subject: [PATCH 41/60] Adjusting dependencies (new locally-built cascading-3.1-wip-dev) + cancel accidental downgrade of scala 2.10 --- project/Build.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 4c3560a776..2da9f309d3 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -53,9 +53,9 @@ object ScaldingBuild extends Build { val sharedSettings = Project.defaultSettings ++ assemblySettings ++ scalariformSettings ++ Seq( organization := "com.twitter", - scalaVersion := "2.10.5", + scalaVersion := "2.10.6", - crossScalaVersions := Seq("2.10.5", "2.11.7"), + crossScalaVersions := Seq("2.10.6", "2.11.7"), ScalariformKeys.preferences := formattingPreferences, @@ -285,10 +285,10 @@ object ScaldingBuild extends Build { lazy val scaldingDate = module("date") lazy val cascadingVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.0.4-wip-dev") // TEMPORARY. This is how a local build ends up named on my machine today -- cchepelov, 2016-01-13 + System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.1.0-wip-dev") // TEMPORARY. This is how a local build ends up named on my machine today -- cchepelov, 2016-01-13 lazy val cascadingJDBCVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "3.0.0-wip-120") + System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "3.0.0-wip-127") lazy val scaldingBenchmarks = module("benchmarks").settings( libraryDependencies ++= Seq( @@ -356,9 +356,10 @@ object ScaldingBuild extends Build { lazy val scaldingParquet = module("parquet").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions - "org.apache.parquet" % "parquet-cascading3" % parquetVersion // FIXME: https://github.com/apache/parquet-mr/pull/284 + "temp.cchepelov.org.apache.parquet" % "parquet-cascading3" % parquetVersion // FIXME: https://github.com/apache/parquet-mr/pull/284 exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-pig") exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-core") + exclude("temp.cchepelov.org.apache.parquet", "parquet-pig") exclude("org.apache.parquet", "parquet-pig") exclude("com.twitter.elephantbird", "elephant-bird-pig") exclude("com.twitter.elephantbird", "elephant-bird-core"), @@ -403,14 +404,15 @@ object ScaldingBuild extends Build { .settings( libraryDependencies ++= Seq( // see https://issues.apache.org/jira/browse/PARQUET-143 for exclusions - "org.apache.parquet" % "parquet-cascading3" % parquetVersion + "temp.cchepelov.org.apache.parquet" % "parquet-cascading3" % parquetVersion exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-pig") exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-core") + exclude("temp.cchepelov.org.apache.parquet", "parquet-pig") exclude("org.apache.parquet", "parquet-pig") exclude("com.twitter.elephantbird", "elephant-bird-pig") exclude("com.twitter.elephantbird", "elephant-bird-core"), "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.apache.parquet" % "parquet-thrift" % parquetVersion % "test" classifier "tests" + "org.apache.parquet" % "parquet-thrift" % "1.8.1" /* FIXME: parquetVersion */ % "test" classifier "tests" exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-pig") exclude("temp.cchepelov.com.twitter.elephantbird", "elephant-bird-core") exclude("org.apache.parquet", "parquet-pig") From 338117ead24436dd0a7e8d5b1aab9cbb8a3436f0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov?= Date: Fri, 15 Jan 2016 14:37:26 +0100 Subject: [PATCH 42/60] per @posco's suggestion, FlowStep[_ <: Any] not FlowStep[_] --- project/Build.scala | 2 +- .../src/main/scala/com/twitter/scalding/Config.scala | 4 ++-- .../main/scala/com/twitter/scalding/ConfigBridge.scala | 10 +++++----- .../scala/com/twitter/scalding/ExecutionContext.scala | 2 +- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 2da9f309d3..f807357e48 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -53,7 +53,7 @@ object ScaldingBuild extends Build { val sharedSettings = Project.defaultSettings ++ assemblySettings ++ scalariformSettings ++ Seq( organization := "com.twitter", - scalaVersion := "2.10.6", + scalaVersion := "2.11.7", crossScalaVersions := Seq("2.10.6", "2.11.7"), diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index fdde19ee03..09ae379969 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -361,8 +361,8 @@ trait Config extends Serializable { /** Get the number of reducers (this is the parameter Hadoop will use) */ def getNumReducers: Option[Int] = get(Config.HadoopNumReducersLegacy) - .orElse(get(Config.HadoopNumReducers2)) - .map(_.toInt) + .orElse(get(Config.HadoopNumReducers2)) + .map(_.toInt) def setNumReducers(n: Int): Config = this + (Config.HadoopNumReducersLegacy -> n.toString) // Note: setting the legacy key for cascading-hadoop compat, hadoop-2.6.0 still accepts it. diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala b/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala index ef3fa86001..a4d071ff71 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ConfigBridge.scala @@ -10,9 +10,9 @@ import scala.collection.JavaConverters._ object ConfigBridge { /** * This adapter handles the various vessel types that can be used to configure properties of a a - * FlowStep[_] since Cascading 3.0 + * FlowStep[_<: Any] since Cascading 3.0 */ - class FlowStepAdapter(flowStep: FlowStep[_]) { + class FlowStepAdapter(flowStep: FlowStep[_ <: Any]) { def getConfigValue(key: String): String = flowStep.getConfig match { case conf: org.apache.hadoop.conf.Configuration => conf.get(key) @@ -21,7 +21,7 @@ object ConfigBridge { case _ => throw new NotImplementedError(s"unknown flowStep Config type ${flowStep.getConfig.getClass}") } - def setConfigValue(name: String, value: String): FlowStep[_] = { + def setConfigValue(name: String, value: String): FlowStep[_ <: Any] = { flowStep.getConfig match { case conf: org.apache.hadoop.conf.Configuration => conf.set(name, value) case conf: org.apache.commons.configuration.Configuration => conf.addProperty(name, value) @@ -34,7 +34,7 @@ object ConfigBridge { def fromPlatform(anyConf: Any): Config = { anyConf match { - /* NOTE: for now we always return a Config instance (actually, an anonymous realization of the Config trait) + /* NOTE: for now we always return a Config instance (actually, an anonymous realization of the Config trait) no matter what the underlying fabric/platform. Here would be a GREAT opportunity to return a specific Config implementation (notably, to deal with things like HadoopNumReducers* ) */ @@ -47,5 +47,5 @@ object ConfigBridge { } } - implicit def toFlowStepPimpFromFlowStep(flowStep: FlowStep[_]): FlowStepAdapter = new FlowStepAdapter(flowStep) + implicit def toFlowStepPimpFromFlowStep(flowStep: FlowStep[_ <: Any]): FlowStepAdapter = new FlowStepAdapter(flowStep) } \ No newline at end of file diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala index 8b185ba31c..44ad574b31 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala @@ -45,7 +45,7 @@ trait ExecutionContext { if (descriptions.nonEmpty) Some(descriptions.distinct.mkString(", ")) else None } - private def updateStepConfigWithDescriptions(step: BaseFlowStep[_]): Unit = { + private def updateStepConfigWithDescriptions(step: BaseFlowStep[_ <: Any]): Unit = { import ConfigBridge._ getIdentifierOpt(ExecutionContext.getDesc(step)) From 24dfb7c4469d39f6cc683a4e78f8d1d7966bf71b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov?= Date: Mon, 18 Jan 2016 08:19:02 +0100 Subject: [PATCH 43/60] RFC: merge of grouped++streamed no longer supported by Cascading 3.0. Alternative tests to resolve this --- .../scalding/platform/PlatformTest.scala | 75 ++++++++++++++++++- 1 file changed, 72 insertions(+), 3 deletions(-) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala index e0345778ce..be791187ae 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala @@ -24,7 +24,7 @@ import com.twitter.scalding._ import com.twitter.scalding.serialization.OrderedSerialization import java.util.{ Iterator => JIterator } import org.scalacheck.{ Arbitrary, Gen } -import org.scalatest.{ Matchers, WordSpec } +import org.scalatest.{ Ignore, Matchers, WordSpec } import org.slf4j.{ LoggerFactory, Logger } import scala.collection.JavaConverters._ import scala.language.experimental.macros @@ -34,6 +34,7 @@ class InAndOutJob(args: Args) extends Job(args) { Tsv("input").read.write(Tsv("output")) } +/*********** RFC SECTION ***********/ object TinyJoinAndMergeJob { val peopleInput = TypedTsv[Int]("input1") val peopleData = List(1, 2, 3, 4) @@ -44,7 +45,6 @@ object TinyJoinAndMergeJob { val output = TypedTsv[(Int, Int)]("output") val outputData = List((1, 2), (2, 2), (3, 2), (4, 1)) } - class TinyJoinAndMergeJob(args: Args) extends Job(args) { import TinyJoinAndMergeJob._ @@ -57,6 +57,47 @@ class TinyJoinAndMergeJob(args: Args) extends Job(args) { (messages ++ people).groupBy('id) { _.size('count) }.write(output) } +class TinyJoinAndMergeJob2(args: Args) extends Job(args) { + import TinyJoinAndMergeJob._ + + val people = peopleInput.read.mapTo(0 -> 'id) { v: Int => v } + + val messages = messageInput.read + .mapTo(0 -> 'id) { v: Int => v } + .joinWithTiny('id -> 'id, people) + .forceToDisk + + (messages ++ people).groupBy('id) { _.size('count) }.write(output) +} + +object TinyJoinAndMergeJob3 { + val peopleInput = TypedTsv[Int]("input1") + val peopleData = List(1, 2, 3, 4) + + val messageInput = TypedTsv[Int]("input2") + val messageData = List(1, 2, 3) + + val peopleInput3 = TypedTsv[Int]("input3") + val peopleData3 = peopleData + + val output = TypedTsv[(Int, Int)]("output") + val outputData = List((1, 2), (2, 2), (3, 2), (4, 1)) +} +class TinyJoinAndMergeJob3(args: Args) extends Job(args) { + import TinyJoinAndMergeJob3._ + + val people = peopleInput.read.mapTo(0 -> 'id) { v: Int => v } + val people3 = peopleInput3.read.mapTo(0 -> 'id) { v: Int => v } + + val messages = messageInput.read + .mapTo(0 -> 'id) { v: Int => v } + .joinWithTiny('id -> 'id, people) + .forceToDisk + + (messages ++ people3).groupBy('id) { _.size('count) }.write(output) +} +/************** END RFC SECTION ************/ + object TsvNoCacheJob { val dataInput = TypedTsv[String]("fakeInput") val data = List("-0.2f -0.3f -0.5f", "-0.1f", "-0.5f") @@ -286,10 +327,12 @@ class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest } } + /********** RFC SECTION **********/ "A TinyJoinAndMergeJob" should { import TinyJoinAndMergeJob._ - "merge and joinWithTiny shouldn't duplicate data" in { + /* FIXME: @cwensel says this leads to an unsupportable query plan, and from Cascading 3.0 this is rejected… */ + "merge and joinWithTiny shouldn't duplicate data" ignore { HadoopPlatformJobTest(new TinyJoinAndMergeJob(_), cluster) .source(peopleInput, peopleData) .source(messageInput, messageData) @@ -298,6 +341,32 @@ class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest } } + "A TinyJoinAndMergeJob" should { + import TinyJoinAndMergeJob._ + + "merge and joinWithTiny shouldn't duplicate data (2)" in { + HadoopPlatformJobTest(new TinyJoinAndMergeJob2(_), cluster) + .source(peopleInput, peopleData) + .source(messageInput, messageData) + .sink(output) { _.toSet shouldBe (outputData.toSet) } + .run + } + } + + "A TinyJoinAndMergeJob" should { + import TinyJoinAndMergeJob3._ + + "merge and joinWithTiny shouldn't duplicate data (3)" in { + HadoopPlatformJobTest(new TinyJoinAndMergeJob3(_), cluster) + .source(peopleInput, peopleData) + .source(messageInput, messageData) + .source(peopleInput3, peopleData3) + .sink(output) { _.toSet shouldBe (outputData.toSet) } + .run + } + } + + /************** END RFC SECTION ***************/ "A TsvNoCacheJob" should { import TsvNoCacheJob._ From 16c4ff8eb28c4dc191cef33ac937c3920a71b2b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov?= Date: Mon, 18 Jan 2016 08:19:51 +0100 Subject: [PATCH 44/60] Using cascading-3.1-wip-47 --- project/Build.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/Build.scala b/project/Build.scala index f807357e48..43ef3e14c8 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -285,7 +285,7 @@ object ScaldingBuild extends Build { lazy val scaldingDate = module("date") lazy val cascadingVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.1.0-wip-dev") // TEMPORARY. This is how a local build ends up named on my machine today -- cchepelov, 2016-01-13 + System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.1.0-wip-47") lazy val cascadingJDBCVersion = System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "3.0.0-wip-127") From 214c59df29e69743152c871a9cf6cb49bbd265d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Mon, 18 Jan 2016 11:14:02 +0100 Subject: [PATCH 45/60] Adjusting expected test line numbers (yearning for C++'s good old __LINE__ macro...) --- .../com/twitter/scalding/platform/PlatformTest.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala index be791187ae..f92ac2e101 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala @@ -419,10 +419,8 @@ class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest val firstStepDescs = steps.headOption.map(_.getConfigValue(Config.StepDescriptions)).getOrElse("") val firstStepDescSet = firstStepDescs.split(",").map(_.trim).toSet - val expected = Set(149, 151, 152, 155, 156).map { i => - s"com.twitter.scalding.platform.TypedPipeJoinWithDescriptionJob.(PlatformTest.scala:$i)" - } ++ Seq("leftJoin", "hashJoin") - + val expected = Set(190, 192, 193, 196, 197).map(linenum => /* WARNING: keep aligned with line numbers above */ + s"com.twitter.scalding.platform.TypedPipeJoinWithDescriptionJob.(PlatformTest.scala:${linenum})") ++ Seq("leftJoin", "hashJoin") firstStepDescSet should equal(expected) steps.map(_.getConfigValue(Config.StepDescriptions)).foreach(s => info(s)) } @@ -438,7 +436,7 @@ class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest val expectedDescs = Set("map stage - assign words to 1", "reduce stage - sum", "write") ++ - Seq(138, 139, 141, 142, 143).map( + Seq(180, 179, 182, 183, 184).map( /* WARNING: keep aligned with line numbers above */ linenum => s"com.twitter.scalding.platform.TypedPipeWithDescriptionJob.(PlatformTest.scala:${linenum})") val foundDescs = steps.map(_.getConfigValue(Config.StepDescriptions).split(",").map(_.trim).toSet) From 15047dc1737c7aa98fff9490a128076b755b3395 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Mon, 18 Jan 2016 12:00:28 +0100 Subject: [PATCH 46/60] bump Hadoop to 2.7.1, pulled in by Cascading anyway --- project/Build.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/Build.scala b/project/Build.scala index 43ef3e14c8..936075fda7 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -28,7 +28,7 @@ object ScaldingBuild extends Build { val chillVersion = "0.7.1" val elephantbirdVersion = "4.11-SNAPSHOT" // https://github.com/twitter/elephant-bird/pull/454 val hadoopLzoVersion = "0.4.19" - val hadoopVersion = "2.5.0" + val hadoopVersion = "2.7.1" val hbaseVersion = "0.94.10" val hravenVersion = "0.9.17.t05" val jacksonVersion = "2.4.2" From a7602d585cd85c12481d6f8e1e19792dacef9946 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Mon, 18 Jan 2016 20:37:27 +0100 Subject: [PATCH 47/60] (WIP: opening up hadoop2-mr1 and tez tests) --- .travis.yml | 16 +++++ project/Build.scala | 44 +++++++++++--- .../scala/com/twitter/scalding/Mode.scala | 60 ++++++++++++++++--- .../src/{test => main}/resources/hipster.txt | 0 .../src/{test => main}/resources/scores.tsv | 0 .../scalding/platform/LocalCluster.scala | 5 +- .../scalding/platform/PlatformTest.scala | 2 +- .../RatioBasedEstimatorTest.scala | 2 +- .../ReducerEstimatorTest.scala | 2 +- .../RuntimeReducerEstimatorTest.scala | 2 +- .../twitter/scalding/HadoopFabricTest.scala | 14 +++++ .../scalding/Hadoop2MR1FabricTest.scala | 14 +++++ .../scalding/Hadoop2TezFabricTest.scala | 33 ++++++++++ .../parquet/tuple/TypedParquetTupleTest.scala | 4 +- 14 files changed, 174 insertions(+), 24 deletions(-) rename scalding-hadoop-test/src/{test => main}/resources/hipster.txt (100%) rename scalding-hadoop-test/src/{test => main}/resources/scores.tsv (100%) rename scalding-hadoop-test/src/{test => main}/scala/com/twitter/scalding/platform/PlatformTest.scala (99%) rename scalding-hadoop-test/src/{test => main}/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala (99%) rename scalding-hadoop-test/src/{test => main}/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala (98%) rename scalding-hadoop-test/src/{test => main}/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala (99%) create mode 100644 scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala create mode 100644 scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala create mode 100644 scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala diff --git a/.travis.yml b/.travis.yml index a9a5a1ec11..8942f39fc0 100644 --- a/.travis.yml +++ b/.travis.yml @@ -46,6 +46,22 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" + - scala: 2.10.6 + env: BUILD="base" TEST_TARGET="scalding-hadoop2-mr1-test" + script: "scripts/run_test.sh" + + - scala: 2.11.7 + env: BUILD="base" TEST_TARGET="scalding-hadoop2-mr1-test" + script: "scripts/run_test.sh" + + - scala: 2.10.6 + env: BUILD="base" TEST_TARGET="scalding-hadoop2-tez-test" + script: "scripts/run_test.sh" + + - scala: 2.11.7 + env: BUILD="base" TEST_TARGET="scalding-hadoop2-tez-test" + script: "scripts/run_test.sh" + - scala: 2.10.6 env: BUILD="base" TEST_TARGET="scalding-serialization" script: "scripts/run_test.sh" diff --git a/project/Build.scala b/project/Build.scala index 936075fda7..4430074b59 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -221,6 +221,8 @@ object ScaldingBuild extends Build { scaldingJson, scaldingJdbc, scaldingHadoopTest, + scaldingHadoop2MR1Test, + scaldingHadoop2TezTest, scaldingDb, maple, executionTutorial, @@ -263,7 +265,7 @@ object ScaldingBuild extends Build { * This returns the youngest jar we released that is compatible with * the current. */ - val unreleasedModules = Set[String]("hadoop-test") //releases 0.11 + val unreleasedModules = Set[String]("hadoop-test", "hadoop2-mr1-test", "hadoop2-tez") //releases 0.11 def youngestForwardCompatible(subProj: String) = Some(subProj) @@ -287,6 +289,9 @@ object ScaldingBuild extends Build { lazy val cascadingVersion = System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.1.0-wip-47") + lazy val tezVersion = + System.getenv.asScala.getOrElse("SCALDING_TEZ_VERSION", "0.6.2") // should match what cascading-hadoop2-tez is using + lazy val cascadingJDBCVersion = System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "3.0.0-wip-127") @@ -302,7 +307,7 @@ object ScaldingBuild extends Build { lazy val scaldingCore = module("core").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "cascading" % "cascading-core" % cascadingVersion, - "cascading" % "cascading-hadoop" % cascadingVersion, // ought to become "provided" or no longer here. + "cascading" % "cascading-hadoop" % cascadingVersion % "provided", "cascading" % "cascading-local" % cascadingVersion, "com.twitter" % "chill-hadoop" % chillVersion, "com.twitter" % "chill-java" % chillVersion, @@ -333,6 +338,7 @@ object ScaldingBuild extends Build { "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-cascading3" % elephantbirdVersion, "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion, "com.hadoop.gplcompression" % "hadoop-lzo" % hadoopLzoVersion, + "cascading" % "cascading-hadoop" % cascadingVersion % "test", // TODO: split this out into scalding-thrift "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "org.apache.thrift" % "libthrift" % thriftVersion, @@ -366,6 +372,8 @@ object ScaldingBuild extends Build { "org.apache.thrift" % "libthrift" % "0.7.0", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", + "cascading" % "cascading-hadoop" % cascadingVersion % "provided", + "cascading" % "cascading-hadoop" % cascadingVersion % "test", "org.scala-lang" % "scala-reflect" % scalaVersion, "com.twitter" %% "bijection-macros" % bijectionVersion, "com.twitter" %% "chill-bijection" % chillVersion @@ -422,12 +430,15 @@ object ScaldingBuild extends Build { "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion, "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-pig" % elephantbirdVersion, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", + "cascading" % "cascading-hadoop" % hadoopVersion % "provided", + "cascading" % "cascading-hadoop" % hadoopVersion % "test", "com.novocode" % "junit-interface" % "0.11" % "test", "junit" % "junit" % junitVersion % "test" ) ).dependsOn(scaldingCore, scaldingParquet % "compile->compile;test->test", scaldingParquetScroogeFixtures % "test->test") + lazy val scaldingHRaven = module("hraven").settings( libraryDependencies ++= Seq( "com.twitter.hraven" % "hraven-core" % hravenVersion @@ -471,6 +482,7 @@ object ScaldingBuild extends Build { "org.scala-lang" % "scala-reflect" % scalaVersion, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "unprovided", + "cascading" % "cascading-hadoop" % cascadingVersion % "provided", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided", "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "unprovided" @@ -500,6 +512,7 @@ object ScaldingBuild extends Build { lazy val scaldingJson = module("json").settings( libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", + "cascading" % "cascading-hadoop" % cascadingVersion % "provided", "com.fasterxml.jackson.module" %% "jackson-module-scala" % jacksonVersion, "org.json4s" %% "json4s-native" % json4SVersion, "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-cascading3" % elephantbirdVersion % "provided" @@ -525,7 +538,8 @@ object ScaldingBuild extends Build { "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion classifier "tests", "org.apache.hadoop" % "hadoop-common" % hadoopVersion classifier "tests", "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % hadoopVersion classifier "tests", - // ought to depend here on "cascading" % "cascading-hadoop" % cascadingVersion + "cascading" % "cascading-hadoop" % cascadingVersion % "provided", + "cascading" % "cascading-hadoop" % cascadingVersion % "test", "com.twitter" %% "chill-algebird" % chillVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, @@ -535,9 +549,21 @@ object ScaldingBuild extends Build { } ).dependsOn(scaldingCore, scaldingSerialization) - /* cchepelov 2016-01-13 Question: create scaldingHadoop2MR1Test, scaldingHadoop2TezTest modules here? Depending on a scalding-hadoop-common ? - * scaldingHadoopTest & scaldingHadoop2MR1Test can probably share the reducer_estimation tests. Much less sure about Tez. - * */ + lazy val scaldingHadoop2MR1Test = module("hadoop2-mr1-test").settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( + "cascading" % "cascading-hadoop2-mr1" % cascadingVersion % "test" + ) } + ).dependsOn(scaldingCore, scaldingSerialization, scaldingHadoopTest) + + lazy val scaldingHadoop2TezTest = module("hadoop2-tez-test").settings( + libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( + "org.apache.tez" % "tez-api" % tezVersion % "test", + "org.apache.tez" % "tez-dag" % tezVersion % "test", + "org.apache.tez" % "tez-mapreduce" % tezVersion % "test", + "org.apache.tez" % "tez-mapreduce" % tezVersion % "test" classifier "tests", + "cascading" % "cascading-hadoop2-tez" % cascadingVersion % "test" + ) } + ).dependsOn(scaldingCore, scaldingSerialization, scaldingHadoopTest) // This one uses a different naming convention @@ -557,7 +583,7 @@ object ScaldingBuild extends Build { libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "org.apache.hbase" % "hbase" % hbaseVersion % "provided", - "cascading" % "cascading-hadoop" % cascadingVersion + "cascading" % "cascading-hadoop" % cascadingVersion % "provided" ) } ) @@ -617,7 +643,9 @@ lazy val scaldingThriftMacros = module("thrift-macros") "org.apache.hadoop" % "hadoop-yarn-server" % hadoopVersion % "test", "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion classifier "tests", "org.apache.hadoop" % "hadoop-common" % hadoopVersion classifier "tests", - "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % hadoopVersion classifier "tests" + "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % hadoopVersion classifier "tests", + "cascading" % "cascading-hadoop" % cascadingVersion % "test", + "cascading" % "cascading-hadoop" % cascadingVersion classifier "tests" ) ++ (if (isScala210x(scalaVersion)) Seq("org.scalamacros" %% "quasiquotes" % "2.0.1") else Seq()) }, addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala index eb70fcdf9a..b14251bcd1 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala @@ -66,14 +66,55 @@ object Mode { val CascadingFlowConnectorClassKey = "cascading.flow.connector.class" val CascadingFlowProcessClassKey = "cascading.flow.process.class" + case class FabricSelector(flowConnectorClassName: String, flowProcessClassName: String) { + def exists = { + try { + val k1 = Class.forName(flowConnectorClassName) + val k2 = Class.forName(flowProcessClassName) + (k1 != null) && (k2 != null) + } catch { + case ncd: ClassNotFoundException => false + } + } + + def selectInto(config: Configuration) = { + config.set(CascadingFlowConnectorClassKey, flowConnectorClassName) + config.set(CascadingFlowProcessClassKey, flowProcessClassName) + } + } + val DefaultHadoopFlowConnector = "cascading.flow.hadoop.HadoopFlowConnector" val DefaultHadoopFlowProcess = "cascading.flow.hadoop.HadoopFlowProcess" + val DefaultHadoopFabric = FabricSelector(DefaultHadoopFlowConnector, DefaultHadoopFlowProcess) val DefaultHadoop2Mr1FlowConnector = "cascading.flow.hadoop2.Hadoop2MR1FlowConnector" val DefaultHadoop2Mr1FlowProcess = "cascading.flow.hadoop.HadoopFlowProcess" // no Hadoop2MR1FlowProcess as of Cascading 3.0.0-wip-75? + val DefaultHadoop2Mr1Fabric = FabricSelector(DefaultHadoop2Mr1FlowConnector, DefaultHadoop2Mr1FlowProcess) val DefaultHadoop2TezFlowConnector = "cascading.flow.tez.Hadoop2TezFlowConnector" val DefaultHadoop2TezFlowProcess = "cascading.flow.tez.Hadoop2TezFlowProcess" + val DefaultHadoop2TezFabric = FabricSelector(DefaultHadoop2TezFlowConnector, DefaultHadoop2TezFlowProcess) + + val DefaultFlinkFlowConnector = "com.dataartisans.flink.cascading.FlinkConnector" + val DefaultFlinkFlowProcess = "com.dataartisans.flink.cascading.runtime.util.FlinkFlowProcess" + val DefaultFlinkFabric = FabricSelector(DefaultFlinkFlowConnector, DefaultFlinkFlowProcess) + + private lazy val selectedFabric = { + val candidates = Seq(DefaultHadoop2TezFabric, DefaultHadoopFabric, DefaultHadoop2Mr1Fabric, DefaultFlinkFabric) + + val selected = candidates.find(_.exists) + if (selected.isEmpty) { + throw new IllegalArgumentException("Can't find a default Cascading fabric. Have you put one in the CLASSPATH?") + } + + LoggerFactory.getLogger(getClass) + .info(s"Using Cascading Flow Connector: ${selected.get.flowConnectorClassName} found in CLASSPATH") + selected.get + } + + def setDefaultFabricFromClasspath(config: Configuration) = { + selectedFabric.selectInto(config) + } // This should be passed ALL the args supplied after the job name def apply(args: Args, config: Configuration): Mode = { @@ -85,22 +126,23 @@ object Mode { if (args.boolean("local")) Local(strictSources) - else if (args.boolean("hdfs")) /* FIXME: should we start printing deprecation warnings ? It's okay to set manually c.f.*.class though */ + else if (args.boolean("hdfs")) { + setDefaultFabricFromClasspath(config) Hdfs(strictSources, config) - else if (args.boolean("hadoop1")) { - config.set(CascadingFlowConnectorClassKey, DefaultHadoopFlowConnector) - config.set(CascadingFlowProcessClassKey, DefaultHadoopFlowProcess) + } else if (args.boolean("hadoop1")) { + DefaultHadoopFabric.selectInto(config) Hdfs(strictSources, config) } else if (args.boolean("hadoop2-mr1")) { - config.set(CascadingFlowConnectorClassKey, DefaultHadoop2Mr1FlowConnector) - config.set(CascadingFlowProcessClassKey, DefaultHadoop2Mr1FlowProcess) + DefaultHadoop2Mr1Fabric.selectInto(config) Hdfs(strictSources, config) } else if (args.boolean("hadoop2-tez")) { - config.set(CascadingFlowConnectorClassKey, DefaultHadoop2TezFlowConnector) - config.set(CascadingFlowProcessClassKey, DefaultHadoop2TezFlowProcess) + DefaultHadoop2TezFabric.selectInto(config) + Hdfs(strictSources, config) + } else if (args.boolean("flink")) { + DefaultFlinkFabric.selectInto(config) Hdfs(strictSources, config) } else - throw ArgsException("[ERROR] Mode must be one of --local, --hadoop1, --hadoop2-mr1, --hadoop2-tez or --hdfs, you provided none") + throw ArgsException("[ERROR] Mode must be one of --local, --hadoop1, --hadoop2-mr1, --hadoop2-tez, --flink or --hdfs, you provided none") } } diff --git a/scalding-hadoop-test/src/test/resources/hipster.txt b/scalding-hadoop-test/src/main/resources/hipster.txt similarity index 100% rename from scalding-hadoop-test/src/test/resources/hipster.txt rename to scalding-hadoop-test/src/main/resources/hipster.txt diff --git a/scalding-hadoop-test/src/test/resources/scores.tsv b/scalding-hadoop-test/src/main/resources/scores.tsv similarity index 100% rename from scalding-hadoop-test/src/test/resources/scores.tsv rename to scalding-hadoop-test/src/main/resources/scores.tsv diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala index 37e03ad95e..bd3155cc9f 100644 --- a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/LocalCluster.scala @@ -171,7 +171,10 @@ class LocalCluster(mutex: Boolean = true) { private def getFileForClass[T](clazz: Class[T]): File = new File(clazz.getProtectionDomain.getCodeSource.getLocation.toURI) - def mode: Mode = Hdfs(true, jobConf) + def mode: Mode = { + Mode.setDefaultFabricFromClasspath(jobConf) + Hdfs(true, jobConf) + } def putFile(file: File, location: String): Boolean = { val hdfsLocation = new Path(location) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala similarity index 99% rename from scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala rename to scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala index f92ac2e101..4d5a98c174 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/platform/PlatformTest.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala @@ -313,7 +313,7 @@ class CheckForFlowProcessInTypedJob(args: Args) extends Job(args) { // Keeping all of the specifications in the same tests puts the result output all together at the end. // This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. -class PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest { +trait PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest { import ConfigBridge._ "An InAndOutTest" should { diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala similarity index 99% rename from scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala rename to scalding-hadoop-test/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala index 7c9a11d806..0bb3e09698 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala @@ -114,7 +114,7 @@ class InvalidHistoryBasedEstimator extends RatioBasedEstimator { override val historyService = InvalidHistoryService } -class RatioBasedReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatformTest { +trait RatioBasedReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatformTest { import HipJob._ "Single-step job with ratio-based reducer estimator" should { diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala similarity index 98% rename from scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala rename to scalding-hadoop-test/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index b0e2974ce2..4032aaa10b 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -84,7 +84,7 @@ class SimpleMapOnlyJob(args: Args, customConfig: Config) extends Job(args) { .write(TypedTsv[String]("mapped_output")) } -class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatformTest { +trait ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatformTest { import HipJob._ "Single-step job with reducer estimator" should { diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala similarity index 99% rename from scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala rename to scalding-hadoop-test/src/main/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala index 5fe85494cd..223ba24b73 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala @@ -34,7 +34,7 @@ class DummyEstimator extends ReducerEstimator { def estimateReducers(info: FlowStrategyInfo) = Some(42) } -class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatformTest { +trait RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatformTest { "Single-step job with runtime-based reducer estimator" should { "set reducers correctly with median estimation scheme" in { diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala new file mode 100644 index 0000000000..7840e5850d --- /dev/null +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala @@ -0,0 +1,14 @@ +package com.twitter.scalding + +import com.twitter.scalding.platform.PlatformTest +import com.twitter.scalding.reducer_estimation.{ RuntimeReducerEstimatorTest, ReducerEstimatorTest, RatioBasedReducerEstimatorTest } + +// Keeping all of the specifications in the same tests puts the result output all together at the end. +// This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. +class HadoopPlatformTest + extends PlatformTest + with RatioBasedReducerEstimatorTest + with ReducerEstimatorTest + with RuntimeReducerEstimatorTest { + /* just realizing here the tests in a Hadooop (1.x API) context, using cascading-hadoop */ +} \ No newline at end of file diff --git a/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala b/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala new file mode 100644 index 0000000000..681b9d54e8 --- /dev/null +++ b/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala @@ -0,0 +1,14 @@ +package com.twitter.scalding + +import com.twitter.scalding.platform.PlatformTest +import com.twitter.scalding.reducer_estimation.{ RuntimeReducerEstimatorTest, ReducerEstimatorTest, RatioBasedReducerEstimatorTest } + +// Keeping all of the specifications in the same tests puts the result output all together at the end. +// This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. +class Hadoop2MR1PlatformTest + extends PlatformTest + with RatioBasedReducerEstimatorTest + with ReducerEstimatorTest + with RuntimeReducerEstimatorTest { + /* just realizing here the tests in a Hadooop+MAPREDUCE (2.x API) context, using cascading-hadoop2-mr1 */ +} diff --git a/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala b/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala new file mode 100644 index 0000000000..a5802d40d7 --- /dev/null +++ b/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala @@ -0,0 +1,33 @@ +package com.twitter.scalding + +import cascading.pipe.assembly.AggregateByProps +import com.twitter.scalding.platform.PlatformTest +import com.twitter.scalding.reducer_estimation.{ RuntimeReducerEstimatorTest, ReducerEstimatorTest, RatioBasedReducerEstimatorTest } +import org.apache.tez.dag.api.TezConfiguration +import cascading.flow.FlowRuntimeProps + +// Keeping all of the specifications in the same tests puts the result output all together at the end. +// This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. +class Hadoop2TezPlatformTest + extends PlatformTest + with RatioBasedReducerEstimatorTest + with ReducerEstimatorTest + with RuntimeReducerEstimatorTest { + /* just realizing here the tests in a Tez context, using cascading-hadoop2-tez */ + + override def initialize(): cluster.type = { + + val tempdir = if (Option(System.getProperty("hadoop.tmp.dir")).getOrElse("").isEmpty) "build/test/tmp" else System.getProperty("hadoop.tmp.dir") + + cluster.initialize(Config.empty + + (TezConfiguration.TEZ_LOCAL_MODE, "true") + + ("tez.runtime.optimize.local.fetch" -> "true") + + ("tez.am.inline.task.execution.max-tasks" -> "3") + + (TezConfiguration.TEZ_IGNORE_LIB_URIS -> "true") + + (TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS -> "true") + + ("tez.am.mode.session" -> "true") // allows multiple TezClient instances to be used in a single jvm + + ("hadoop.tmp.dir" -> tempdir) + + ("mapred.mapper.new-api" -> "true") /* mandatory to use Tez */ + + (cascading.flow.FlowRuntimeProps.GATHER_PARTITIONS -> "4") /* a value must be provided */ ) + } +} diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala index e7f0eb4764..d9efaa00a1 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala @@ -1,9 +1,9 @@ package com.twitter.scalding.parquet.tuple import com.twitter.scalding.parquet.tuple.macros.Macros._ -import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopPlatformTest } +import com.twitter.scalding.platform.HadoopPlatformJobTest import com.twitter.scalding.typed.TypedPipe -import com.twitter.scalding.{ Args, Job, TypedTsv } +import com.twitter.scalding.{HadoopPlatformTest, Args, Job, TypedTsv} import org.scalatest.{ Matchers, WordSpec } import org.apache.parquet.filter2.predicate.FilterApi.binaryColumn import org.apache.parquet.filter2.predicate.{ FilterApi, FilterPredicate } From b752423c434f94d1f8d6ed900df46c9251cf7f94 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov?= Date: Mon, 18 Jan 2016 23:00:26 +0100 Subject: [PATCH 48/60] (WIP; fixing some compile errors) --- .../scala/com/twitter/scalding/HadoopFabricTest.scala | 2 +- .../scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala | 2 +- .../scala/com/twitter/scalding/Hadoop2TezFabricTest.scala | 8 ++++---- .../scalding/parquet/tuple/TypedParquetTupleTest.scala | 6 +++--- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala index 7840e5850d..4962cd435b 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala @@ -5,7 +5,7 @@ import com.twitter.scalding.reducer_estimation.{ RuntimeReducerEstimatorTest, Re // Keeping all of the specifications in the same tests puts the result output all together at the end. // This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. -class HadoopPlatformTest +class HadoopFabricTest extends PlatformTest with RatioBasedReducerEstimatorTest with ReducerEstimatorTest diff --git a/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala b/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala index 681b9d54e8..414d92f779 100644 --- a/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala +++ b/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala @@ -5,7 +5,7 @@ import com.twitter.scalding.reducer_estimation.{ RuntimeReducerEstimatorTest, Re // Keeping all of the specifications in the same tests puts the result output all together at the end. // This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. -class Hadoop2MR1PlatformTest +class Hadoop2MR1FabricTest extends PlatformTest with RatioBasedReducerEstimatorTest with ReducerEstimatorTest diff --git a/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala b/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala index a5802d40d7..3528f9e725 100644 --- a/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala +++ b/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala @@ -8,7 +8,7 @@ import cascading.flow.FlowRuntimeProps // Keeping all of the specifications in the same tests puts the result output all together at the end. // This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. -class Hadoop2TezPlatformTest +class Hadoop2TezFabricTest extends PlatformTest with RatioBasedReducerEstimatorTest with ReducerEstimatorTest @@ -22,12 +22,12 @@ class Hadoop2TezPlatformTest cluster.initialize(Config.empty + (TezConfiguration.TEZ_LOCAL_MODE, "true") + ("tez.runtime.optimize.local.fetch" -> "true") - + ("tez.am.inline.task.execution.max-tasks" -> "3") + + (TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS -> "3") + (TezConfiguration.TEZ_IGNORE_LIB_URIS -> "true") + (TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS -> "true") - + ("tez.am.mode.session" -> "true") // allows multiple TezClient instances to be used in a single jvm + + (TezConfiguration.TEZ_AM_SESSION_MODE -> "true") // allows multiple TezClient instances to be used in a single jvm + ("hadoop.tmp.dir" -> tempdir) - + ("mapred.mapper.new-api" -> "true") /* mandatory to use Tez */ + // + ("mapred.mapper.new-api" -> "true") /* mandatory to use Tez */ + (cascading.flow.FlowRuntimeProps.GATHER_PARTITIONS -> "4") /* a value must be provided */ ) } } diff --git a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala index d9efaa00a1..3b43fad6c3 100644 --- a/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala +++ b/scalding-parquet/src/test/scala/com/twitter/scalding/parquet/tuple/TypedParquetTupleTest.scala @@ -1,15 +1,15 @@ package com.twitter.scalding.parquet.tuple import com.twitter.scalding.parquet.tuple.macros.Macros._ -import com.twitter.scalding.platform.HadoopPlatformJobTest +import com.twitter.scalding.platform.{ PlatformTest, HadoopPlatformJobTest } import com.twitter.scalding.typed.TypedPipe -import com.twitter.scalding.{HadoopPlatformTest, Args, Job, TypedTsv} +import com.twitter.scalding.{ Args, Job, TypedTsv } import org.scalatest.{ Matchers, WordSpec } import org.apache.parquet.filter2.predicate.FilterApi.binaryColumn import org.apache.parquet.filter2.predicate.{ FilterApi, FilterPredicate } import org.apache.parquet.io.api.Binary -class TypedParquetTupleTest extends WordSpec with Matchers with HadoopPlatformTest { +class TypedParquetTupleTest extends WordSpec with Matchers with PlatformTest { "TypedParquetTuple" should { "read and write correctly" in { From f1000949a0e0c706e9f55b12ea406d40581a2462 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 19 Jan 2016 10:46:28 +0100 Subject: [PATCH 49/60] Disable tests which will not work in Tez --- .../scala/com/twitter/scalding/Hadoop2TezFabricTest.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala b/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala index 3528f9e725..b078b0ce13 100644 --- a/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala +++ b/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala @@ -9,10 +9,9 @@ import cascading.flow.FlowRuntimeProps // Keeping all of the specifications in the same tests puts the result output all together at the end. // This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. class Hadoop2TezFabricTest - extends PlatformTest - with RatioBasedReducerEstimatorTest + extends PlatformTest /*with RatioBasedReducerEstimatorTest with ReducerEstimatorTest - with RuntimeReducerEstimatorTest { + with RuntimeReducerEstimatorTest */ { /* just realizing here the tests in a Tez context, using cascading-hadoop2-tez */ override def initialize(): cluster.type = { From 62de186713544c2c7e593d821e77b0c45c68e06d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 19 Jan 2016 10:47:06 +0100 Subject: [PATCH 50/60] Revisit fabric-switching logic to also work in Mode#openForRead (required by tests) --- .../scala/com/twitter/scalding/Mode.scala | 59 ++++++++++++++----- 1 file changed, 44 insertions(+), 15 deletions(-) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala index b14251bcd1..e0d9181996 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala @@ -65,39 +65,46 @@ object Mode { val CascadingFlowConnectorClassKey = "cascading.flow.connector.class" val CascadingFlowProcessClassKey = "cascading.flow.process.class" + val CascadingFlowProcessConfigClassKey = "cascading.flow.process.config.class" - case class FabricSelector(flowConnectorClassName: String, flowProcessClassName: String) { + case class FabricSelector(flowConnectorClassName: String, flowProcessClassName: String, flowProcessConfigClassName: String) { def exists = { try { val k1 = Class.forName(flowConnectorClassName) val k2 = Class.forName(flowProcessClassName) - (k1 != null) && (k2 != null) + val k3 = Class.forName(flowProcessConfigClassName) + (k1 != null) && (k2 != null) && (k3 != null) } catch { - case ncd: ClassNotFoundException => false + case cnfe: ClassNotFoundException => false } } def selectInto(config: Configuration) = { config.set(CascadingFlowConnectorClassKey, flowConnectorClassName) config.set(CascadingFlowProcessClassKey, flowProcessClassName) + config.set(CascadingFlowProcessConfigClassKey, flowProcessConfigClassName) } } val DefaultHadoopFlowConnector = "cascading.flow.hadoop.HadoopFlowConnector" val DefaultHadoopFlowProcess = "cascading.flow.hadoop.HadoopFlowProcess" - val DefaultHadoopFabric = FabricSelector(DefaultHadoopFlowConnector, DefaultHadoopFlowProcess) + val DefaultHadoopFlowProcessConfig = "org.apache.hadoop.mapred.JobConf" + val DefaultHadoopFabric = FabricSelector(DefaultHadoopFlowConnector, DefaultHadoopFlowProcess, DefaultHadoopFlowProcessConfig) val DefaultHadoop2Mr1FlowConnector = "cascading.flow.hadoop2.Hadoop2MR1FlowConnector" - val DefaultHadoop2Mr1FlowProcess = "cascading.flow.hadoop.HadoopFlowProcess" // no Hadoop2MR1FlowProcess as of Cascading 3.0.0-wip-75? - val DefaultHadoop2Mr1Fabric = FabricSelector(DefaultHadoop2Mr1FlowConnector, DefaultHadoop2Mr1FlowProcess) + val DefaultHadoop2Mr1FlowProcess = DefaultHadoopFlowProcess // no Hadoop2MR1FlowProcess as of Cascading 3.0.0-wip-75? + val DefaultHadoop2Mr1FlowProcessConfig = DefaultHadoopFlowProcessConfig + val DefaultHadoop2Mr1Fabric = FabricSelector(DefaultHadoop2Mr1FlowConnector, DefaultHadoop2Mr1FlowProcess, DefaultHadoop2Mr1FlowProcessConfig) val DefaultHadoop2TezFlowConnector = "cascading.flow.tez.Hadoop2TezFlowConnector" val DefaultHadoop2TezFlowProcess = "cascading.flow.tez.Hadoop2TezFlowProcess" - val DefaultHadoop2TezFabric = FabricSelector(DefaultHadoop2TezFlowConnector, DefaultHadoop2TezFlowProcess) + val DefaultHadoop2TezFlowProcessConfig = "org.apache.tez.dag.api.TezConfiguration" + val DefaultHadoop2TezFabric = FabricSelector(DefaultHadoop2TezFlowConnector, DefaultHadoop2TezFlowProcess, DefaultHadoop2TezFlowProcessConfig) val DefaultFlinkFlowConnector = "com.dataartisans.flink.cascading.FlinkConnector" val DefaultFlinkFlowProcess = "com.dataartisans.flink.cascading.runtime.util.FlinkFlowProcess" - val DefaultFlinkFabric = FabricSelector(DefaultFlinkFlowConnector, DefaultFlinkFlowProcess) + val DefaultFlinkFlowProcessConfiguration = "org.apache.hadoop.conf.Configuration" + val DefaultFlinkFabric = FabricSelector(DefaultFlinkFlowConnector, DefaultFlinkFlowProcess, DefaultFlinkFlowProcessConfiguration) private lazy val selectedFabric = { val candidates = Seq(DefaultHadoop2TezFabric, DefaultHadoopFabric, DefaultHadoop2Mr1Fabric, DefaultFlinkFabric) @@ -198,17 +205,39 @@ trait HadoopMode extends Mode { // TODO unlike newFlowConnector, this does not look at the Job.config override def openForRead(config: Config, tap: Tap[_, _, _]) = { - val htap = tap.asInstanceOf[Tap[JobConf, _, _]] - val conf = new JobConf(true) // initialize the default config - // copy over Config - config.toMap.foreach{ case (k, v) => conf.set(k, v) } + val htap: Tap[Configuration, _, _] = tap.asInstanceOf[Tap[Configuration, _, _]] val flowProcessClass = jobConf.get(Mode.CascadingFlowProcessClassKey, Mode.DefaultHadoopFlowProcess) + val flowProcessConfigClass = jobConf.get(Mode.CascadingFlowProcessConfigClassKey, Mode.DefaultHadoopFlowProcessConfig) - val fp = try { + val (fp, conf) = try { val clazz = Class.forName(flowProcessClass) - val ctor = clazz.getConstructor(classOf[JobConf]) - ctor.newInstance(conf).asInstanceOf[FlowProcess[JobConf]] + val confClazz = Class.forName(flowProcessConfigClass) + if (!classOf[Configuration].isAssignableFrom(confClazz)) { + throw new IllegalArgumentException(s"FlowProcess configuration type ${confClazz} does not implement ${classOf[Configuration]}") + } + + + val conf = { + try { + /* first constructor attempted: supposed to accept a Boolean where "true" means "load system defaults" */ + val confCtor = confClazz.getConstructor(java.lang.Boolean.TYPE) + confCtor.newInstance(java.lang.Boolean.TRUE).asInstanceOf[Configuration] + } catch { + case _ : NoSuchMethodError | _ : NoSuchMethodException => { + /* fallback: the Configuration should have a default constructor */ + val confCtor = confClazz.getConstructor() + confCtor.newInstance().asInstanceOf[Configuration] + } + } + } // initialize the default config + // copy over Config + config.toMap.foreach{ case (k, v) => conf.set(k, v) } + + + val ctor = clazz.getConstructor(confClazz) + val inst = ctor.newInstance(conf) + (inst.asInstanceOf[FlowProcess[_ <: Configuration]], conf) } catch { case ncd: ClassNotFoundException => { throw new ModeLoadException("Failed to load Cascading flow process class " + flowProcessClass, ncd) From 8eec3bf660443b9429bb40251923214cb68d2d6e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 19 Jan 2016 10:48:06 +0100 Subject: [PATCH 51/60] The Tez fabric can't determine whether Maple tap/inputformats content to the old or new API. Work around that. --- .../scala/com/twitter/scalding/Hadoop2TezFabricTest.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala b/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala index b078b0ce13..a3757b5835 100644 --- a/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala +++ b/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala @@ -26,7 +26,11 @@ class Hadoop2TezFabricTest + (TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS -> "true") + (TezConfiguration.TEZ_AM_SESSION_MODE -> "true") // allows multiple TezClient instances to be used in a single jvm + ("hadoop.tmp.dir" -> tempdir) - // + ("mapred.mapper.new-api" -> "true") /* mandatory to use Tez */ + + ("mapred.mapper.new-api" -> { + if (classOf[org.apache.hadoop.mapred.InputFormat[_, _]].isAssignableFrom(classOf[com.twitter.maple.tap.TupleMemoryInputFormat])) "false" + else if (classOf[org.apache.hadoop.mapreduce.InputFormat[_, _]].isAssignableFrom(classOf[com.twitter.maple.tap.TupleMemoryInputFormat])) "true" + else ??? + }) /* we are using c.t.maple.tap.MemorySourceTap, which Cascading can't identify as being in the old or new API */ + (cascading.flow.FlowRuntimeProps.GATHER_PARTITIONS -> "4") /* a value must be provided */ ) } } From c39c6b3ac1dcbc436d2fc75b7b0eae14481d163d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 19 Jan 2016 10:53:15 +0100 Subject: [PATCH 52/60] PlatformTest.IterableSource test: remove incorrect implicit ordering assumption (which is not upheld in Tez) --- .../scala/com/twitter/scalding/platform/PlatformTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala index 4d5a98c174..2598bd49fd 100644 --- a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala @@ -455,19 +455,19 @@ trait PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest "distinct properly from normal data" in { HadoopPlatformJobTest(new NormalDistinctJob(_), cluster) .source[String]("input", data ++ data ++ data) - .sink[String]("output") { _.toList shouldBe data } + .sink[String]("output") { _.toList.sorted shouldBe data.sorted } // important: don't do set comparisons or "distinct" issues might be missed .run } "distinctBy(identity) properly from a list in memory" in { HadoopPlatformJobTest(new IterableSourceDistinctIdentityJob(_), cluster) - .sink[String]("output") { _.toList shouldBe data } + .sink[String]("output") { _.toList.sorted shouldBe data.sorted } // important: don't do set comparisons or "distinct" issues might be missed .run } "distinct properly from a list" in { HadoopPlatformJobTest(new IterableSourceDistinctJob(_), cluster) - .sink[String]("output") { _.toList shouldBe data } + .sink[String]("output") { _.toList.sorted shouldBe data.sorted } // important: don't do set comparisons or "distinct" issues might be missed .run } } From fd4d7f4b2f95e519f764e012d1ded88f0aa3b813 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 19 Jan 2016 11:48:16 +0100 Subject: [PATCH 53/60] Ensure at least SOME test also run on Tez --- .gitignore | 3 +- .../scala/com/twitter/scalding/Mode.scala | 4 +- .../platform/DagwisePlatformTest.scala | 51 +++++++++++++++++++ .../scalding/platform/PlatformTest.scala | 19 ------- .../platform/StepwisePlatformTest.scala | 47 +++++++++++++++++ .../twitter/scalding/HadoopFabricTest.scala | 4 +- .../scalding/Hadoop2MR1FabricTest.scala | 4 +- .../scalding/Hadoop2TezFabricTest.scala | 4 +- 8 files changed, 107 insertions(+), 29 deletions(-) create mode 100644 scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/DagwisePlatformTest.scala create mode 100644 scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/StepwisePlatformTest.scala diff --git a/.gitignore b/.gitignore index 0182e6b9aa..2bffe0b01b 100644 --- a/.gitignore +++ b/.gitignore @@ -35,4 +35,5 @@ tutorial/data/avrooutput0.avro scalding-hadoop-test/NOTICE NOTICE *~ -build/ \ No newline at end of file +build/ +.staging \ No newline at end of file diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala index e0d9181996..3d2b77c69d 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Mode.scala @@ -217,14 +217,13 @@ trait HadoopMode extends Mode { throw new IllegalArgumentException(s"FlowProcess configuration type ${confClazz} does not implement ${classOf[Configuration]}") } - val conf = { try { /* first constructor attempted: supposed to accept a Boolean where "true" means "load system defaults" */ val confCtor = confClazz.getConstructor(java.lang.Boolean.TYPE) confCtor.newInstance(java.lang.Boolean.TRUE).asInstanceOf[Configuration] } catch { - case _ : NoSuchMethodError | _ : NoSuchMethodException => { + case _: NoSuchMethodError | _: NoSuchMethodException => { /* fallback: the Configuration should have a default constructor */ val confCtor = confClazz.getConstructor() confCtor.newInstance().asInstanceOf[Configuration] @@ -234,7 +233,6 @@ trait HadoopMode extends Mode { // copy over Config config.toMap.foreach{ case (k, v) => conf.set(k, v) } - val ctor = clazz.getConstructor(confClazz) val inst = ctor.newInstance(conf) (inst.asInstanceOf[FlowProcess[_ <: Configuration]], conf) diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/DagwisePlatformTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/DagwisePlatformTest.scala new file mode 100644 index 0000000000..6ad3db2f6c --- /dev/null +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/DagwisePlatformTest.scala @@ -0,0 +1,51 @@ +package com.twitter.scalding.platform + +import cascading.flow.FlowStep +import cascading.flow.planner.BaseFlowStep +import cascading.pipe.joiner.{ JoinerClosure, InnerJoin } +import cascading.tuple.Tuple + +import com.twitter.scalding._ +import com.twitter.scalding.serialization.OrderedSerialization +import java.util.{ Iterator => JIterator } +import org.scalacheck.{ Arbitrary, Gen } +import org.scalatest.{ Ignore, Matchers, WordSpec } +import org.slf4j.{ LoggerFactory, Logger } +import scala.collection.JavaConverters._ +import scala.language.experimental.macros +import scala.math.Ordering + +/** + * This trait includes tests which only make sense on platforms where each Flow is translated + * as a single DAG onto the underlying execution fabric + * (e.g. Tez, Flink but possibly not Spark) + * + */ +trait DagwisePlatformTest extends PlatformTest { + import ConfigBridge._ + + "A TypedPipeForceToDiskWithDescriptionPipe" should { + "have a custom step name from withDescription" in { + + HadoopPlatformJobTest(new TypedPipeForceToDiskWithDescriptionJob(_), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val labels = steps.head.getConfigValue(Config.StepDescriptions).split(",").map(_.trim).toSet + labels.contains("write words to disk") should be(true) + labels.contains("output frequency by length") should be(true) + + /* ".forceToDisk" may have an influence on Tez and other "whole DAG" processing engines but + it should not cause a new Step + + (note: "partial DAG" engines where Cascading has to generate 1 or more steps + depending on the Flow's exact topology may disagree about this; in which case + make another trait from PlatformTest to model this and put the appropriate tests) + */ + } + .run + } + } + +} diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala index 2598bd49fd..f6790b1b0b 100644 --- a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/PlatformTest.scala @@ -391,25 +391,6 @@ trait PlatformTest extends WordSpec with Matchers with HadoopSharedPlatformTest } - "A TypedPipeForceToDiskWithDescriptionPipe" should { - "have a custom step name from withDescription" in { - - HadoopPlatformJobTest(new TypedPipeForceToDiskWithDescriptionJob(_), cluster) - .inspectCompletedFlow { flow => - val steps = flow.getFlowSteps.asScala - val firstStep = steps.filter(_.getName.startsWith("(1/2)")) - val secondStep = steps.filter(_.getName.startsWith("(2/2)")) - val lab1 = firstStep.map(_.getConfigValue(Config.StepDescriptions)) - lab1 should have size 1 - lab1(0) should include ("write words to disk") - val lab2 = secondStep.map(_.getConfigValue(Config.StepDescriptions)) - lab2 should have size 1 - lab2(0) should include ("output frequency by length") - } - .run - } - } - "A TypedPipeJoinWithDescriptionPipe" should { "have a custom step name from withDescription" in { HadoopPlatformJobTest(new TypedPipeJoinWithDescriptionJob(_), cluster) diff --git a/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/StepwisePlatformTest.scala b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/StepwisePlatformTest.scala new file mode 100644 index 0000000000..f774272ad3 --- /dev/null +++ b/scalding-hadoop-test/src/main/scala/com/twitter/scalding/platform/StepwisePlatformTest.scala @@ -0,0 +1,47 @@ +package com.twitter.scalding.platform + +import cascading.flow.FlowStep +import cascading.flow.planner.BaseFlowStep +import cascading.pipe.joiner.{ JoinerClosure, InnerJoin } +import cascading.tuple.Tuple + +import com.twitter.scalding._ +import com.twitter.scalding.serialization.OrderedSerialization +import java.util.{ Iterator => JIterator } +import org.scalacheck.{ Arbitrary, Gen } +import org.scalatest.{ Ignore, Matchers, WordSpec } +import org.slf4j.{ LoggerFactory, Logger } +import scala.collection.JavaConverters._ +import scala.language.experimental.macros +import scala.math.Ordering + +/** + * This trait includes tests which only make sense on platforms where each (or most) Cascading + * step is translated as an individual job on the underlying execution fabric + * (e.g. Hadoop MAPREDUCE) + * + * + */ +trait StepwisePlatformTest extends PlatformTest { + import ConfigBridge._ + + "A TypedPipeForceToDiskWithDescriptionPipe" should { + "have a custom step name from withDescription" in { + + HadoopPlatformJobTest(new TypedPipeForceToDiskWithDescriptionJob(_), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + val firstStep = steps.filter(_.getName.startsWith("(1/2)")) + val secondStep = steps.filter(_.getName.startsWith("(2/2)")) + val lab1 = firstStep.map(_.getConfigValue(Config.StepDescriptions)) + lab1 should have size 1 + lab1(0) should include ("write words to disk") + val lab2 = secondStep.map(_.getConfigValue(Config.StepDescriptions)) + lab2 should have size 1 + lab2(0) should include ("output frequency by length") + } + .run + } + } + +} diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala index 4962cd435b..e7b3794dcf 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/HadoopFabricTest.scala @@ -1,12 +1,12 @@ package com.twitter.scalding -import com.twitter.scalding.platform.PlatformTest +import com.twitter.scalding.platform.StepwisePlatformTest import com.twitter.scalding.reducer_estimation.{ RuntimeReducerEstimatorTest, ReducerEstimatorTest, RatioBasedReducerEstimatorTest } // Keeping all of the specifications in the same tests puts the result output all together at the end. // This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. class HadoopFabricTest - extends PlatformTest + extends StepwisePlatformTest with RatioBasedReducerEstimatorTest with ReducerEstimatorTest with RuntimeReducerEstimatorTest { diff --git a/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala b/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala index 414d92f779..a931390efe 100644 --- a/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala +++ b/scalding-hadoop2-mr1-test/src/test/scala/com/twitter/scalding/Hadoop2MR1FabricTest.scala @@ -1,12 +1,12 @@ package com.twitter.scalding -import com.twitter.scalding.platform.PlatformTest +import com.twitter.scalding.platform.StepwisePlatformTest import com.twitter.scalding.reducer_estimation.{ RuntimeReducerEstimatorTest, ReducerEstimatorTest, RatioBasedReducerEstimatorTest } // Keeping all of the specifications in the same tests puts the result output all together at the end. // This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. class Hadoop2MR1FabricTest - extends PlatformTest + extends StepwisePlatformTest with RatioBasedReducerEstimatorTest with ReducerEstimatorTest with RuntimeReducerEstimatorTest { diff --git a/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala b/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala index a3757b5835..f463360f44 100644 --- a/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala +++ b/scalding-hadoop2-tez-test/src/test/scala/com/twitter/scalding/Hadoop2TezFabricTest.scala @@ -1,7 +1,7 @@ package com.twitter.scalding import cascading.pipe.assembly.AggregateByProps -import com.twitter.scalding.platform.PlatformTest +import com.twitter.scalding.platform.DagwisePlatformTest import com.twitter.scalding.reducer_estimation.{ RuntimeReducerEstimatorTest, ReducerEstimatorTest, RatioBasedReducerEstimatorTest } import org.apache.tez.dag.api.TezConfiguration import cascading.flow.FlowRuntimeProps @@ -9,7 +9,7 @@ import cascading.flow.FlowRuntimeProps // Keeping all of the specifications in the same tests puts the result output all together at the end. // This is useful given that the Hadoop MiniMRCluster and MiniDFSCluster spew a ton of logging. class Hadoop2TezFabricTest - extends PlatformTest /*with RatioBasedReducerEstimatorTest + extends DagwisePlatformTest /*with RatioBasedReducerEstimatorTest with ReducerEstimatorTest with RuntimeReducerEstimatorTest */ { /* just realizing here the tests in a Tez context, using cascading-hadoop2-tez */ From e861b08623d2a3ac9c68eb3ec2eefe989e22f361 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 19 Jan 2016 12:27:55 +0100 Subject: [PATCH 54/60] Reduce the amount of logging noise from Tez during tests --- .../src/test/resources/log4j.properties | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 scalding-hadoop2-tez-test/src/test/resources/log4j.properties diff --git a/scalding-hadoop2-tez-test/src/test/resources/log4j.properties b/scalding-hadoop2-tez-test/src/test/resources/log4j.properties new file mode 100644 index 0000000000..d4e857c374 --- /dev/null +++ b/scalding-hadoop2-tez-test/src/test/resources/log4j.properties @@ -0,0 +1,19 @@ + +# log4j configuration used during build and unit tests + +log4j.rootLogger=info,stdout +log4j.threshold=ALL +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n + +log4j.appender.CLA=org.apache.log4j.AsyncAppender +log4j.appender.CLA.AppenderRef=stdout + + +log4j.logger.cascading=INFO +log4j.logger.org.apache.hadoop=WARN +log4j.logger.org.apache.tez=WARN +log4j.logger.BlockStateChange=WARN +log4j.logger.SecurityLogger=WARN +log4j.logger.org.mortbay=WARN \ No newline at end of file From 734c282d78611420ff37a39ddd545c5b6aae263c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 19 Jan 2016 12:29:19 +0100 Subject: [PATCH 55/60] Add cascading-hadoop back to the assembly used by REPL --- project/Build.scala | 7 ++++--- scripts/scald.rb | 11 +++++++---- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 4430074b59..ff501d5ad2 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -118,7 +118,7 @@ object ScaldingBuild extends Build { testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, "-oF"), // Uncomment if you don't want to run all the tests before building assembly - // test in assembly := {}, + test in assembly := {}, logLevel in assembly := Level.Warn, // Publishing options: @@ -237,7 +237,7 @@ object ScaldingBuild extends Build { ).settings( test := {}, publish := {}, // skip publishing for this root project. - publishLocal := {} + publishLocal := {} ).aggregate( scaldingArgs, scaldingDate, @@ -482,7 +482,8 @@ object ScaldingBuild extends Build { "org.scala-lang" % "scala-reflect" % scalaVersion, "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "unprovided", - "cascading" % "cascading-hadoop" % cascadingVersion % "provided", + "cascading" % "cascading-hadoop" % cascadingVersion /* % "provided" */, /* need to have a fabric around, can't just leave it to "provided" */ + "cascading" % "cascading-hadoop" % cascadingVersion % "unprovided", /* for 'sbt run' */ "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "provided", "org.slf4j" % "slf4j-log4j12" % slf4jVersion % "unprovided" diff --git a/scripts/scald.rb b/scripts/scald.rb index 62dc767c2b..45d201a580 100755 --- a/scripts/scald.rb +++ b/scripts/scald.rb @@ -262,7 +262,10 @@ def get_dep_location(org, dep, version) if (!CONFIG["jar"]) #what jar has all the dependencies for this job - CONFIG["jar"] = repo_root + "/scalding-core/target/scala-#{SHORT_SCALA_VERSION}/scalding-core-assembly-#{SCALDING_VERSION}.jar" + CONFIG["jar"] = repo_root + "/scalding-core/target/scala-#{SHORT_SCALA_VERSION}/scalding-core-assembly-#{SCALDING_VERSION}.jar" + EXTRACP = repo_root + "/scalding-repl/target/scala-#{SHORT_SCALA_VERSION}/scalding-repl-assembly-#{SCALDING_VERSION}.jar" +else + EXTRACP = "" end #Check that we can find the jar: @@ -508,7 +511,7 @@ def needs_rebuild? def build_job_jar $stderr.puts("compiling " + JOBFILE) FileUtils.mkdir_p(BUILDDIR) - classpath = (([LIBCP, JARPATH, MODULEJARPATHS, CLASSPATH].select { |s| s != "" }) + convert_dependencies_to_jars).flatten.join(":") + classpath = (([LIBCP, JARPATH, MODULEJARPATHS, EXTRACP, CLASSPATH].select { |s| s != "" }) + convert_dependencies_to_jars).flatten.join(":") puts("#{file_type}c -classpath #{classpath} -d #{BUILDDIR} #{JOBFILE}") unless system("#{COMPILE_CMD} -classpath #{classpath} -d #{BUILDDIR} #{JOBFILE}") puts "[SUGGESTION]: Try scald.rb --clean, you may have corrupt jars lying around" @@ -523,7 +526,7 @@ def build_job_jar end def hadoop_classpath - (["/usr/share/java/hadoop-lzo-0.4.15.jar", JARBASE, MODULEJARPATHS.map{|n| File.basename(n)}, "job-jars/#{JOBJAR}"].select { |s| s != "" }).flatten.join(":") + (["/usr/share/java/hadoop-lzo-0.4.15.jar", JARBASE, EXTRACP, MODULEJARPATHS.map{|n| File.basename(n)}, "job-jars/#{JOBJAR}"].select { |s| s != "" }).flatten.join(":") end def hadoop_command @@ -570,7 +573,7 @@ def local_cmd(mode) [] end - classpath = ([JARPATH, MODULEJARPATHS].select { |s| s != "" } + convert_dependencies_to_jars + localHadoopDepPaths).flatten.join(":") + (is_file? ? ":#{JOBJARPATH}" : "") + + classpath = ([JARPATH, EXTRACP, MODULEJARPATHS].select { |s| s != "" } + convert_dependencies_to_jars + localHadoopDepPaths).flatten.join(":") + (is_file? ? ":#{JOBJARPATH}" : "") + ":" + CLASSPATH "java -Xmx#{LOCALMEM} -cp #{classpath} #{TOOL} #{JOB} #{mode} #{JOB_ARGS}" end From 4189f699f983603e7e5ee496bb576182da554d53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 20 Jan 2016 11:45:38 +0100 Subject: [PATCH 56/60] tracking Cascading 3.1-wip-48 --- project/Build.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/Build.scala b/project/Build.scala index ff501d5ad2..b1e460148a 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -287,7 +287,7 @@ object ScaldingBuild extends Build { lazy val scaldingDate = module("date") lazy val cascadingVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.1.0-wip-47") + System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.1.0-wip-48") lazy val tezVersion = System.getenv.asScala.getOrElse("SCALDING_TEZ_VERSION", "0.6.2") // should match what cascading-hadoop2-tez is using From 9dcbd7f64ff52d4d597c4eb525aa95c536f984f8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Wed, 20 Jan 2016 11:45:50 +0100 Subject: [PATCH 57/60] more test output into .gitignore --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 2bffe0b01b..bfe900179a 100644 --- a/.gitignore +++ b/.gitignore @@ -31,6 +31,7 @@ tutorial/data/rightDiff.tsv tutorial/data/tmp3.tsv tutorial/data/jsonoutput0.tsv tutorial/data/avrooutput0.avro +tutorial/data/execution_output.txt .scalding_repl scalding-hadoop-test/NOTICE NOTICE From b228b73c1bcb46a2ef1196a51ff8b9c6f01facc5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Thu, 21 Jan 2016 22:24:56 +0100 Subject: [PATCH 58/60] (-commons: post-merge with upstream/develop, lost a "provided" dep on cascading-hadoop) --- project/Build.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/Build.scala b/project/Build.scala index b1e460148a..753057d4f4 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -338,6 +338,7 @@ object ScaldingBuild extends Build { "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-cascading3" % elephantbirdVersion, "temp.cchepelov.com.twitter.elephantbird" % "elephant-bird-core" % elephantbirdVersion, "com.hadoop.gplcompression" % "hadoop-lzo" % hadoopLzoVersion, + "cascading" % "cascading-hadoop" % cascadingVersion % "provided", "cascading" % "cascading-hadoop" % cascadingVersion % "test", // TODO: split this out into scalding-thrift "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", From a69177486a22f0078ea5440a51e2e8ccbb4490a2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Thu, 21 Jan 2016 23:33:35 +0100 Subject: [PATCH 59/60] re-C3'ing the VersionedTap/KeyValueByteScheme that were absorbed from dfs-datastores --- .../commons/scheme/KeyValueByteScheme.java | 6 +-- .../scalding/commons/tap/VersionedTap.java | 47 +++++++++++-------- 2 files changed, 30 insertions(+), 23 deletions(-) diff --git a/scalding-commons/src/main/java/com/twitter/scalding/commons/scheme/KeyValueByteScheme.java b/scalding-commons/src/main/java/com/twitter/scalding/commons/scheme/KeyValueByteScheme.java index bf95160efb..4a7b8d3974 100644 --- a/scalding-commons/src/main/java/com/twitter/scalding/commons/scheme/KeyValueByteScheme.java +++ b/scalding-commons/src/main/java/com/twitter/scalding/commons/scheme/KeyValueByteScheme.java @@ -3,6 +3,7 @@ import java.io.IOException; import java.util.Arrays; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; @@ -29,7 +30,7 @@ public static byte[] getBytes(BytesWritable key) { } @Override - public boolean source(FlowProcess flowProcess, + public boolean source(FlowProcess flowProcess, SourceCall sourceCall) throws IOException { BytesWritable key = (BytesWritable) sourceCall.getContext()[0]; BytesWritable value = (BytesWritable) sourceCall.getContext()[1]; @@ -47,7 +48,7 @@ public boolean source(FlowProcess flowProcess, } @Override - public void sink(FlowProcess flowProcess, SinkCall sinkCall) + public void sink(FlowProcess flowProcess, SinkCall sinkCall) throws IOException { TupleEntry tupleEntry = sinkCall.getOutgoingEntry(); @@ -57,4 +58,3 @@ public void sink(FlowProcess flowProcess, SinkCall scheme, TapMode mode) + public VersionedTap(String dir, Scheme scheme, TapMode mode) throws IOException { super(scheme, dir); this.mode = mode; @@ -59,11 +60,11 @@ public String getOutputDirectory() { return getPath().toString(); } - public VersionedStore getStore(JobConf conf) throws IOException { + public VersionedStore getStore(Configuration conf) throws IOException { return new VersionedStore(FileSystem.get(conf), getOutputDirectory()); } - public String getSourcePath(JobConf conf) { + public String getSourcePath(Configuration conf) { VersionedStore store; try { store = getStore(conf); @@ -77,7 +78,7 @@ public String getSourcePath(JobConf conf) { } } - public String getSinkPath(JobConf conf) { + public String getSinkPath(Configuration conf) { try { VersionedStore store = getStore(conf); String sinkPath = (version == null) ? store.createVersion() : store.createVersion(version); @@ -91,33 +92,37 @@ public String getSinkPath(JobConf conf) { } @Override - public void sourceConfInit(FlowProcess process, JobConf conf) { - super.sourceConfInit(process, conf); - FileInputFormat.setInputPaths(conf, getSourcePath(conf)); + public void sourceConfInit(FlowProcess process, Configuration conf) { + JobConf jobConf = new JobConf(conf); + + super.sourceConfInit(process, jobConf); + FileInputFormat.setInputPaths(jobConf, getSourcePath(jobConf)); } @Override - public void sinkConfInit(FlowProcess process, JobConf conf) { - super.sinkConfInit(process, conf); + public void sinkConfInit(FlowProcess process, Configuration conf) { + JobConf jobConf = new JobConf(conf); + + super.sinkConfInit(process, jobConf); if (newVersionPath == null) - newVersionPath = getSinkPath(conf); + newVersionPath = getSinkPath(jobConf); - FileOutputFormat.setOutputPath(conf, new Path(newVersionPath)); + FileOutputFormat.setOutputPath(jobConf, new Path(newVersionPath)); } @Override - public boolean resourceExists(JobConf jc) throws IOException { + public boolean resourceExists(Configuration jc) throws IOException { return getStore(jc).mostRecentVersion() != null; } @Override - public boolean createResource(JobConf jc) throws IOException { + public boolean createResource(Configuration jc) throws IOException { throw new UnsupportedOperationException("Not supported yet."); } @Override - public boolean deleteResource(JobConf jc) throws IOException { + public boolean deleteResource(Configuration jc) throws IOException { throw new UnsupportedOperationException("Not supported yet."); } @@ -131,18 +136,20 @@ public String getIdentifier() { } @Override - public long getModifiedTime(JobConf conf) throws IOException { + public long getModifiedTime(Configuration conf) throws IOException { VersionedStore store = getStore(conf); return (mode == TapMode.SINK) ? 0 : store.mostRecentVersion(); } @Override - public boolean commitResource(JobConf conf) throws IOException { - VersionedStore store = new VersionedStore(FileSystem.get(conf), getOutputDirectory()); + public boolean commitResource(Configuration conf) throws IOException { + JobConf jobConf = new JobConf(conf); + + VersionedStore store = new VersionedStore(FileSystem.get(jobConf), getOutputDirectory()); if (newVersionPath != null) { store.succeedVersion(newVersionPath); - markSuccessfulOutputDir(new Path(newVersionPath), conf); + markSuccessfulOutputDir(new Path(newVersionPath), jobConf); newVersionPath = null; store.cleanup(getVersionsToKeep()); } @@ -150,7 +157,7 @@ public boolean commitResource(JobConf conf) throws IOException { return true; } - private static void markSuccessfulOutputDir(Path path, JobConf conf) throws IOException { + private static void markSuccessfulOutputDir(Path path, Configuration conf) throws IOException { FileSystem fs = FileSystem.get(conf); // create a file in the folder to mark it if (fs.exists(path)) { @@ -160,7 +167,7 @@ private static void markSuccessfulOutputDir(Path path, JobConf conf) throws IOEx } @Override - public boolean rollbackResource(JobConf conf) throws IOException { + public boolean rollbackResource(Configuration conf) throws IOException { if (newVersionPath != null) { getStore(conf).failVersion(newVersionPath); newVersionPath = null; From eb7546f38db4c50aa399d7148aa616916c0b3796 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov=20=28TP12=29?= Date: Tue, 26 Jan 2016 18:30:11 +0100 Subject: [PATCH 60/60] (follow with Cascading, 3.1-wip-52 + tez 0.8.2; and with Scalding/develop) --- project/Build.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/project/Build.scala b/project/Build.scala index 753057d4f4..12c85d91ce 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -287,10 +287,10 @@ object ScaldingBuild extends Build { lazy val scaldingDate = module("date") lazy val cascadingVersion = - System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.1.0-wip-48") + System.getenv.asScala.getOrElse("SCALDING_CASCADING_VERSION", "3.1.0-wip-52") lazy val tezVersion = - System.getenv.asScala.getOrElse("SCALDING_TEZ_VERSION", "0.6.2") // should match what cascading-hadoop2-tez is using + System.getenv.asScala.getOrElse("SCALDING_TEZ_VERSION", "0.8.2") // should match what cascading-hadoop2-tez is using lazy val cascadingJDBCVersion = System.getenv.asScala.getOrElse("SCALDING_CASCADING_JDBC_VERSION", "3.0.0-wip-127") @@ -585,6 +585,7 @@ object ScaldingBuild extends Build { libraryDependencies <++= (scalaVersion) { scalaVersion => Seq( "org.apache.hadoop" % "hadoop-client" % hadoopVersion % "provided", "org.apache.hbase" % "hbase" % hbaseVersion % "provided", + "cascading" % "cascading-core" % cascadingVersion, "cascading" % "cascading-hadoop" % cascadingVersion % "provided" ) }