/*
 * Copyright (c) 2010-2015 Pivotal Software, Inc. All rights reserved.
 *
 * 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. See accompanying
 * LICENSE file.
 */
package com.pivotal.gemfirexd.hadoop.mapred;

import java.io.IOException;
import java.lang.reflect.Method;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.List;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.mapred.InvalidJobConfException;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.OutputFormat;
import org.apache.hadoop.mapred.RecordWriter;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.util.Progressable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import com.pivotal.gemfirexd.hadoop.mapreduce.Key;
import com.pivotal.gemfirexd.internal.engine.hadoop.mapreduce.OutputFormatUtil;
import com.pivotal.gemfirexd.internal.engine.hadoop.mapreduce.OutputFormatUtil.RowCommandBatchExecutor;

import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;

/**
 * Map Reduce {@link OutputFormat} implementation for Gemfirexd. This class uses
 * client connection to populate records in the DB. User is responsible for
 * knowing name and type of the table columns. User provides a data class which
 * contains setter methods for each column which needs to be populated. This
 * class calls each setter method to populate the insert query.
 * 
 * @author ashvina
 * @param <VALUE>
 *          User's data class containing setter methods for table columns
 */
@SuppressFBWarnings(
    value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
    justification = "IO format classes in hadoop world seem to follow this convention for new and old MR apis.")
public class RowOutputFormat<VALUE> extends
    com.pivotal.gemfirexd.hadoop.mapreduce.RowOutputFormat<VALUE> implements
    OutputFormat<Key, VALUE> {
  
  /**
   * The name of the table to output to, for example APP.CUSTOMERS. This should
   * match the table name used with the CREATE TABLE statement.
   */
  public static final String OUTPUT_TABLE = OutputFormatUtil.OUTPUT_TABLE;
  /**
   * JDBC url used to connect to a running gemfirexd instance. For example
   * jdbc:gemfirexd://myHostName:1527/
   */
  public static final String OUTPUT_URL = OutputFormatUtil.OUTPUT_URL;
  
  /**
   * By default row insert commands are executed in batch mode. The batch size
   * can be configured using this property. The default value is 10K commands.
   * If the batch size is equal or less than 0, then the batch is when close
   * operation is executed
   */
  public static final String OUTPUT_BATCH_SIZE = com.pivotal.gemfirexd.hadoop.mapreduce.RowOutputFormat.OUTPUT_BATCH_SIZE;

  private OutputFormatUtil util = new OutputFormatUtil();
  private final Logger logger;

  public RowOutputFormat() {
    this.logger = LoggerFactory.getLogger(RowOutputFormat.class);
  }

  /**
   * {@inheritDoc}
   */
  @Override
  public void checkOutputSpecs(FileSystem fs, JobConf job) throws IOException {
    validateConfiguration(job);
  }

  /**
   * {@inheritDoc}
   */
  @Override
  public RecordWriter<Key, VALUE> getRecordWriter(FileSystem fs, JobConf job,
      String name, Progressable progress) throws IOException {
    // TODO progress
    return new MapRedGfxdRecordWriter(job);
  }

  /**
   * {@link RecordWriter} implementation for Gemfirexd. The class uses reflection
   * to identify setter methods in the user provided data class.
   * 
   * @author ashvina
   */
  public class MapRedGfxdRecordWriter implements RecordWriter<Key, VALUE> {
    final private RowCommandBatchExecutor batchExecutor;

    // list of setters in the {@code VALUE} class
    List<Method> columnSetters = new ArrayList<Method>();
    private String tableName;

    public MapRedGfxdRecordWriter(Configuration conf) throws IOException {
      this.tableName = conf.get(OUTPUT_TABLE);
      try {
        this.batchExecutor = util.new RowCommandBatchExecutor(getDriver(conf),
            conf.get(OUTPUT_URL), conf.getInt(OUTPUT_BATCH_SIZE,
                OUTPUT_BATCH_SIZE_DEFAULT));
      } catch (ClassNotFoundException e) {
        logger.error("Gemfirexd client classes are missing from the classpath", e);
        throw new InvalidJobConfException(e);
      }
    }

    /**
     * {@inheritDoc}
     */
    @Override
    public void write(Key key, VALUE value) throws IOException {
      try {
        /**
         * The statement depends on column setters provided by the user. These
         * are not known until first instance of the user's data class is
         * provided. Hence query creation will begin on the first invocation
         */
        if (batchExecutor.isNotInitialized()) {
          // list of setters in the {@code VALUE} class
          columnSetters = util.spotTableColumnSetters(value);
          String query = util.createQuery(tableName, columnSetters);
          logger.debug("Query to be executed by record writer is: " + query);
          batchExecutor.initStatement(query);
        }

        batchExecutor.executeWriteStatement(value, columnSetters);
      } catch (SQLException e) {
        logger.error("Failed to upload data into Gemfirexd", e);
        throw new IOException(e);
      }
    }

    /**
     * {@inheritDoc}
     */
    @Override
    public void close(Reporter reporter) throws IOException {
      // TODO reporter update
      batchExecutor.close();
    }
  }
}