public void computeMatrixCharacteristics(
      MatrixCharacteristics mc1,
      MatrixCharacteristics mc2,
      MatrixCharacteristics mc3,
      MatrixCharacteristics dimOut) {
    QuaternaryOperator qop = (QuaternaryOperator) optr;

    if (qop.wtype1 != null || qop.wtype4 != null) { // wsloss/wcemm
      // output size independent of chain type (scalar)
      dimOut.set(1, 1, mc1.getRowsPerBlock(), mc1.getColsPerBlock());
    } else if (qop.wtype2 != null || qop.wtype5 != null) { // wsigmoid/wumm
      // output size determined by main input
      dimOut.set(mc1.getRows(), mc1.getCols(), mc1.getRowsPerBlock(), mc1.getColsPerBlock());
    } else if (qop.wtype3 != null) { // wdivmm
      // note: cannot directly consume mc2 or mc3 for redwdivmm because rep instruction changed
      // the relevant dimensions; as a workaround the original dims are passed via nnz
      boolean mapwdivmm = _cacheU && _cacheV;
      long rank =
          qop.wtype3.isLeft()
              ? mapwdivmm ? mc3.getCols() : mc3.getNonZeros()
              : mapwdivmm ? mc2.getCols() : mc2.getNonZeros();
      MatrixCharacteristics mcTmp =
          qop.wtype3.computeOutputCharacteristics(mc1.getRows(), mc1.getCols(), rank);
      dimOut.set(mcTmp.getRows(), mcTmp.getCols(), mc1.getRowsPerBlock(), mc1.getColsPerBlock());
    }
  }
  @Override
  public void processInstruction(ExecutionContext ec)
      throws DMLRuntimeException, DMLUnsupportedOperationException {
    SparkExecutionContext sec = (SparkExecutionContext) ec;
    MatrixCharacteristics mc = sec.getMatrixCharacteristics(input1.getName());
    long rlen = mc.getRows();
    int brlen = mc.getRowsPerBlock();
    int bclen = mc.getColsPerBlock();

    // get input
    JavaPairRDD<MatrixIndexes, MatrixBlock> in =
        sec.getBinaryBlockRDDHandleForVariable(input1.getName());

    // execute unary aggregate (w/ implicit drop correction)
    AggregateUnaryOperator auop = (AggregateUnaryOperator) _optr;
    JavaPairRDD<MatrixIndexes, MatrixBlock> out =
        in.mapToPair(new RDDCumAggFunction(auop, rlen, brlen, bclen));
    out = RDDAggregateUtils.mergeByKey(out);

    // put output handle in symbol table
    sec.setRDDHandleForVariable(output.getName(), out);
    sec.addLineageRDD(output.getName(), input1.getName());
  }
 /**
  * @param type
  * @param mcBc
  * @return
  */
 private static boolean requiresFlatMapFunction(CacheType type, MatrixCharacteristics mcBc) {
   return (type == CacheType.LEFT && mcBc.getRows() > mcBc.getRowsPerBlock())
       || (type == CacheType.RIGHT && mcBc.getCols() > mcBc.getColsPerBlock());
 }
 /**
  * @param mcIn
  * @param type
  * @return
  */
 private static boolean preservesPartitioning(MatrixCharacteristics mcIn, CacheType type) {
   if (type == CacheType.LEFT) return mcIn.dimsKnown() && mcIn.getRows() <= mcIn.getRowsPerBlock();
   else // RIGHT
   return mcIn.dimsKnown() && mcIn.getCols() <= mcIn.getColsPerBlock();
 }
  /**
   * @param pfid
   * @param program
   * @param taskFile
   * @param resultFile
   * @param _enableCPCaching
   * @param mode
   * @param numMappers
   * @param replication
   * @return
   * @throws DMLRuntimeException
   */
  public static RemoteParForJobReturn runJob(
      long pfid,
      String program,
      String taskFile,
      String resultFile,
      MatrixObject colocatedDPMatrixObj, // inputs
      boolean enableCPCaching,
      int numMappers,
      int replication,
      int max_retry,
      long minMem,
      boolean jvmReuse) // opt params
      throws DMLRuntimeException {
    RemoteParForJobReturn ret = null;
    String jobname = "ParFor-EMR";
    long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0;

    JobConf job;
    job = new JobConf(RemoteParForMR.class);
    job.setJobName(jobname + pfid);

    // maintain dml script counters
    Statistics.incrementNoOfCompiledMRJobs();

    try {
      /////
      // configure the MR job

      // set arbitrary CP program blocks that will perform in the mapper
      MRJobConfiguration.setProgramBlocks(job, program);

      // enable/disable caching
      MRJobConfiguration.setParforCachingConfig(job, enableCPCaching);

      // set mappers, reducers, combiners
      job.setMapperClass(RemoteParWorkerMapper.class); // map-only

      // set input format (one split per row, NLineInputFormat default N=1)
      if (ParForProgramBlock.ALLOW_DATA_COLOCATION && colocatedDPMatrixObj != null) {
        job.setInputFormat(RemoteParForColocatedNLineInputFormat.class);
        MRJobConfiguration.setPartitioningFormat(job, colocatedDPMatrixObj.getPartitionFormat());
        MatrixCharacteristics mc = colocatedDPMatrixObj.getMatrixCharacteristics();
        MRJobConfiguration.setPartitioningBlockNumRows(job, mc.getRowsPerBlock());
        MRJobConfiguration.setPartitioningBlockNumCols(job, mc.getColsPerBlock());
        MRJobConfiguration.setPartitioningFilename(job, colocatedDPMatrixObj.getFileName());
      } else // default case
      {
        job.setInputFormat(NLineInputFormat.class);
      }

      // set the input path and output path
      FileInputFormat.setInputPaths(job, new Path(taskFile));

      // set output format
      job.setOutputFormat(SequenceFileOutputFormat.class);

      // set output path
      MapReduceTool.deleteFileIfExistOnHDFS(resultFile);
      FileOutputFormat.setOutputPath(job, new Path(resultFile));

      // set the output key, value schema
      job.setMapOutputKeyClass(LongWritable.class);
      job.setMapOutputValueClass(Text.class);
      job.setOutputKeyClass(LongWritable.class);
      job.setOutputValueClass(Text.class);

      //////
      // set optimization parameters

      // set the number of mappers and reducers
      job.setNumMapTasks(numMappers); // numMappers
      job.setNumReduceTasks(0);
      // job.setInt("mapred.map.tasks.maximum", 1); //system property
      // job.setInt("mapred.tasktracker.tasks.maximum",1); //system property
      // job.setInt("mapred.jobtracker.maxtasks.per.job",1); //system property

      // use FLEX scheduler configuration properties
      if (ParForProgramBlock.USE_FLEX_SCHEDULER_CONF) {
        job.setInt("flex.priority", 0); // highest

        job.setInt("flex.map.min", 0);
        job.setInt("flex.map.max", numMappers);
        job.setInt("flex.reduce.min", 0);
        job.setInt("flex.reduce.max", numMappers);
      }

      // set jvm memory size (if require)
      String memKey = "mapred.child.java.opts";
      if (minMem > 0 && minMem > InfrastructureAnalyzer.extractMaxMemoryOpt(job.get(memKey))) {
        InfrastructureAnalyzer.setMaxMemoryOpt(job, memKey, minMem);
        LOG.warn("Forcing '" + memKey + "' to -Xmx" + minMem / (1024 * 1024) + "M.");
      }

      // disable automatic tasks timeouts and speculative task exec
      job.setInt("mapred.task.timeout", 0);
      job.setMapSpeculativeExecution(false);

      // set up map/reduce memory configurations (if in AM context)
      DMLConfig config = ConfigurationManager.getConfig();
      DMLAppMasterUtils.setupMRJobRemoteMaxMemory(job, config);

      // enables the reuse of JVMs (multiple tasks per MR task)
      if (jvmReuse) job.setNumTasksToExecutePerJvm(-1); // unlimited

      // set sort io buffer (reduce unnecessary large io buffer, guaranteed memory consumption)
      job.setInt(MRConfigurationNames.MR_TASK_IO_SORT_MB, 8); // 8MB

      // set the replication factor for the results
      job.setInt("dfs.replication", replication);

      // set the max number of retries per map task
      //  disabled job-level configuration to respect cluster configuration
      //  note: this refers to hadoop2, hence it never had effect on mr1
      // job.setInt("mapreduce.map.maxattempts", max_retry);

      // set unique working dir
      MRJobConfiguration.setUniqueWorkingDir(job);

      /////
      // execute the MR job
      RunningJob runjob = JobClient.runJob(job);

      // Process different counters
      Statistics.incrementNoOfExecutedMRJobs();
      Group pgroup = runjob.getCounters().getGroup(ParForProgramBlock.PARFOR_COUNTER_GROUP_NAME);
      int numTasks = (int) pgroup.getCounter(Stat.PARFOR_NUMTASKS.toString());
      int numIters = (int) pgroup.getCounter(Stat.PARFOR_NUMITERS.toString());
      if (DMLScript.STATISTICS && !InfrastructureAnalyzer.isLocalMode()) {
        Statistics.incrementJITCompileTime(pgroup.getCounter(Stat.PARFOR_JITCOMPILE.toString()));
        Statistics.incrementJVMgcCount(pgroup.getCounter(Stat.PARFOR_JVMGC_COUNT.toString()));
        Statistics.incrementJVMgcTime(pgroup.getCounter(Stat.PARFOR_JVMGC_TIME.toString()));
        Group cgroup =
            runjob.getCounters().getGroup(CacheableData.CACHING_COUNTER_GROUP_NAME.toString());
        CacheStatistics.incrementMemHits(
            (int) cgroup.getCounter(CacheStatistics.Stat.CACHE_HITS_MEM.toString()));
        CacheStatistics.incrementFSBuffHits(
            (int) cgroup.getCounter(CacheStatistics.Stat.CACHE_HITS_FSBUFF.toString()));
        CacheStatistics.incrementFSHits(
            (int) cgroup.getCounter(CacheStatistics.Stat.CACHE_HITS_FS.toString()));
        CacheStatistics.incrementHDFSHits(
            (int) cgroup.getCounter(CacheStatistics.Stat.CACHE_HITS_HDFS.toString()));
        CacheStatistics.incrementFSBuffWrites(
            (int) cgroup.getCounter(CacheStatistics.Stat.CACHE_WRITES_FSBUFF.toString()));
        CacheStatistics.incrementFSWrites(
            (int) cgroup.getCounter(CacheStatistics.Stat.CACHE_WRITES_FS.toString()));
        CacheStatistics.incrementHDFSWrites(
            (int) cgroup.getCounter(CacheStatistics.Stat.CACHE_WRITES_HDFS.toString()));
        CacheStatistics.incrementAcquireRTime(
            cgroup.getCounter(CacheStatistics.Stat.CACHE_TIME_ACQR.toString()));
        CacheStatistics.incrementAcquireMTime(
            cgroup.getCounter(CacheStatistics.Stat.CACHE_TIME_ACQM.toString()));
        CacheStatistics.incrementReleaseTime(
            cgroup.getCounter(CacheStatistics.Stat.CACHE_TIME_RLS.toString()));
        CacheStatistics.incrementExportTime(
            cgroup.getCounter(CacheStatistics.Stat.CACHE_TIME_EXP.toString()));
      }

      // read all files of result variables and prepare for return
      LocalVariableMap[] results = readResultFile(job, resultFile);

      ret = new RemoteParForJobReturn(runjob.isSuccessful(), numTasks, numIters, results);
    } catch (Exception ex) {
      throw new DMLRuntimeException(ex);
    } finally {
      // remove created files
      try {
        MapReduceTool.deleteFileIfExistOnHDFS(new Path(taskFile), job);
        MapReduceTool.deleteFileIfExistOnHDFS(new Path(resultFile), job);
      } catch (IOException ex) {
        throw new DMLRuntimeException(ex);
      }
    }

    if (DMLScript.STATISTICS) {
      long t1 = System.nanoTime();
      Statistics.maintainCPHeavyHitters("MR-Job_" + jobname, t1 - t0);
    }

    return ret;
  }