@Override public void initialize() { super.initialize(); for (PType pt : ptypes) { pt.initialize(getConfiguration()); } }
/** * Created a detached value for a {@link PGroupedTable} value. * * @param groupedTableType The grouped table type * @param value The value from which a detached value is to be created * @return The detached value * @see PType#getDetachedValue(Object) */ public static <K, V> Pair<K, Iterable<V>> getGroupedDetachedValue( PGroupedTableType<K, V> groupedTableType, Pair<K, Iterable<V>> value) { PTableType<K, V> tableType = groupedTableType.getTableType(); List<V> detachedIterable = Lists.newArrayList(); PType<V> valueType = tableType.getValueType(); for (V v : value.second()) { detachedIterable.add(valueType.getDetachedValue(v)); } return Pair.of( tableType.getKeyType().getDetachedValue(value.first()), (Iterable<V>) detachedIterable); }
private void monitorLoop() { status.set(Status.RUNNING); long start = System.currentTimeMillis(); Map<PCollectionImpl<?>, Set<Target>> targetDeps = Maps.newTreeMap(DEPTH_COMPARATOR); Set<Target> unfinished = Sets.newHashSet(); for (PCollectionImpl<?> pcollect : outputTargets.keySet()) { targetDeps.put(pcollect, pcollect.getTargetDependencies()); unfinished.addAll(outputTargets.get(pcollect)); } runCallables(unfinished); while (!targetDeps.isEmpty() && doneSignal.getCount() > 0) { Set<Target> allTargets = Sets.newHashSet(); for (PCollectionImpl<?> pcollect : targetDeps.keySet()) { allTargets.addAll(outputTargets.get(pcollect)); } Map<PCollectionImpl<?>, JavaRDDLike<?, ?>> pcolToRdd = Maps.newTreeMap(DEPTH_COMPARATOR); for (PCollectionImpl<?> pcollect : targetDeps.keySet()) { if (Sets.intersection(allTargets, targetDeps.get(pcollect)).isEmpty()) { JavaRDDLike<?, ?> rdd = ((SparkCollection) pcollect).getJavaRDDLike(this); pcolToRdd.put(pcollect, rdd); } } distributeFiles(); for (Map.Entry<PCollectionImpl<?>, JavaRDDLike<?, ?>> e : pcolToRdd.entrySet()) { JavaRDDLike<?, ?> rdd = e.getValue(); PType<?> ptype = e.getKey().getPType(); Set<Target> targets = outputTargets.get(e.getKey()); if (targets.size() > 1) { rdd.rdd().cache(); } for (Target t : targets) { Configuration conf = new Configuration(getConfiguration()); getRuntimeContext().setConf(sparkContext.broadcast(WritableUtils.toByteArray(conf))); if (t instanceof MapReduceTarget) { // TODO: check this earlier Converter c = t.getConverter(ptype); IdentityFn ident = IdentityFn.getInstance(); JavaPairRDD<?, ?> outRDD; if (rdd instanceof JavaRDD) { outRDD = ((JavaRDD) rdd) .map( new MapFunction( c.applyPTypeTransforms() ? ptype.getOutputMapFn() : ident, ctxt)) .mapToPair(new OutputConverterFunction(c)); } else { outRDD = ((JavaPairRDD) rdd) .map( new PairMapFunction( c.applyPTypeTransforms() ? ptype.getOutputMapFn() : ident, ctxt)) .mapToPair(new OutputConverterFunction(c)); } try { Job job = new Job(conf); if (t instanceof PathTarget) { PathTarget pt = (PathTarget) t; pt.configureForMapReduce(job, ptype, pt.getPath(), "out0"); CrunchOutputs.OutputConfig outConfig = CrunchOutputs.getNamedOutputs(job.getConfiguration()).get("out0"); job.setOutputFormatClass(outConfig.bundle.getFormatClass()); job.setOutputKeyClass(outConfig.keyClass); job.setOutputValueClass(outConfig.valueClass); outConfig.bundle.configure(job.getConfiguration()); Path tmpPath = pipeline.createTempPath(); outRDD.saveAsNewAPIHadoopFile( tmpPath.toString(), c.getKeyClass(), c.getValueClass(), job.getOutputFormatClass(), job.getConfiguration()); pt.handleOutputs(job.getConfiguration(), tmpPath, -1); } else { // if (t instanceof MapReduceTarget) { MapReduceTarget mrt = (MapReduceTarget) t; mrt.configureForMapReduce(job, ptype, new Path("/tmp"), "out0"); CrunchOutputs.OutputConfig outConfig = CrunchOutputs.getNamedOutputs(job.getConfiguration()).get("out0"); job.setOutputFormatClass(outConfig.bundle.getFormatClass()); job.setOutputKeyClass(outConfig.keyClass); job.setOutputValueClass(outConfig.valueClass); outRDD.saveAsHadoopDataset(new JobConf(job.getConfiguration())); } } catch (Exception et) { LOG.error("Spark Exception", et); status.set(Status.FAILED); set(PipelineResult.EMPTY); doneSignal.countDown(); } } } unfinished.removeAll(targets); } if (status.get() == Status.RUNNING) { for (PCollectionImpl<?> output : pcolToRdd.keySet()) { if (toMaterialize.containsKey(output)) { MaterializableIterable mi = toMaterialize.get(output); if (mi.isSourceTarget()) { output.materializeAt((SourceTarget) mi.getSource()); } } targetDeps.remove(output); } } runCallables(unfinished); } if (status.get() != Status.FAILED || status.get() != Status.KILLED) { status.set(Status.SUCCEEDED); set( new PipelineResult( ImmutableList.of( new PipelineResult.StageResult( "Spark", getCounters(), start, System.currentTimeMillis())), Status.SUCCEEDED)); } else { set(PipelineResult.EMPTY); } doneSignal.countDown(); }