/** It represents one region server as one split. */
public class TableSplit implements Serializable, Writable {
  private static final long serialVersionUID = -8058151330863145575L;

  private static final LogService LOGGER =
      LogServiceFactory.getLogService(TableSplit.class.getName());
  private List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);

  private Partition partition;

  /** @return the locations */
  public List<String> getLocations() {
    return locations;
  }

  /** @param locations the locations to set */
  public void setLocations(List<String> locations) {
    this.locations = locations;
  }

  /** @return Returns the partitions. */
  public Partition getPartition() {
    return partition;
  }

  /** @param partition The partitions to set. */
  public void setPartition(Partition partition) {
    this.partition = partition;
  }

  @Override
  public void readFields(DataInput in) throws IOException {

    int sizeLoc = in.readInt();
    for (int i = 0; i < sizeLoc; i++) {
      byte[] b = new byte[in.readInt()];
      in.readFully(b);
      locations.add(new String(b, Charset.defaultCharset()));
    }

    byte[] buf = new byte[in.readInt()];
    in.readFully(buf);
    ByteArrayInputStream bis = new ByteArrayInputStream(buf);
    ObjectInputStream ois = new ObjectInputStream(bis);
    try {
      partition = (Partition) ois.readObject();
    } catch (ClassNotFoundException e) {
      LOGGER.error(e, e.getMessage());
    }
    ois.close();
  }

  @Override
  public void write(DataOutput out) throws IOException {

    int sizeLoc = locations.size();
    out.writeInt(sizeLoc);
    for (int i = 0; i < sizeLoc; i++) {
      byte[] bytes = locations.get(i).getBytes(Charset.defaultCharset());
      out.writeInt(bytes.length);
      out.write(bytes);
    }

    ByteArrayOutputStream bos = new ByteArrayOutputStream();

    ObjectOutputStream obs = new ObjectOutputStream(bos);
    obs.writeObject(partition);
    obs.close();
    byte[] byteArray = bos.toByteArray();
    out.writeInt(byteArray.length);
    out.write(byteArray);
  }

  public String toString() {
    return partition.getUniqueID() + ' ' + locations;
  }
}
/**
 * In case of detail query we cannot keep all the records in memory so for executing that query are
 * returning a iterator over block and every time next call will come it will execute the block and
 * return the result
 */
public abstract class AbstractDetailQueryResultIterator extends CarbonIterator {

  /** LOGGER. */
  private static final LogService LOGGER =
      LogServiceFactory.getLogService(AbstractDetailQueryResultIterator.class.getName());

  /** execution info of the block */
  protected List<BlockExecutionInfo> blockExecutionInfos;

  /** executor which will execute the query */
  protected InternalQueryExecutor executor;

  /** number of cores which can be used */
  private long numberOfCores;

  /** keep track of number of blocklet per block */
  private long[] totalNumberBlockletPerSlice;

  /** total number of blocklet to be executed */
  private long totalNumberOfNode;

  /** current counter to check how blocklet has been executed */
  protected long currentCounter;

  /** keep the track of number of blocklet of a block has been executed */
  private long[] numberOfBlockletExecutedPerBlock;

  /** block index to be executed */
  protected int[] blockIndexToBeExecuted;

  public AbstractDetailQueryResultIterator(
      List<BlockExecutionInfo> infos,
      QueryExecutorProperties executerProperties,
      QueryModel queryModel,
      InternalQueryExecutor queryExecutor) {
    int recordSize = 0;
    String defaultInMemoryRecordsSize =
        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.INMEMORY_REOCRD_SIZE);
    if (null != defaultInMemoryRecordsSize) {
      try {
        recordSize = Integer.parseInt(defaultInMemoryRecordsSize);
      } catch (NumberFormatException ne) {
        LOGGER.error("Invalid inmemory records size. Using default value");
        recordSize = CarbonCommonConstants.INMEMORY_REOCRD_SIZE_DEFAULT;
      }
    }
    this.numberOfCores =
        recordSize
            / Integer.parseInt(
                CarbonProperties.getInstance()
                    .getProperty(
                        CarbonCommonConstants.BLOCKLET_SIZE,
                        CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
    if (numberOfCores == 0) {
      numberOfCores++;
    }
    executor = queryExecutor;
    this.blockExecutionInfos = infos;
    this.blockIndexToBeExecuted = new int[(int) numberOfCores];
    intialiseInfos();
  }

  private void intialiseInfos() {
    this.totalNumberBlockletPerSlice = new long[blockExecutionInfos.size()];
    this.numberOfBlockletExecutedPerBlock = new long[blockExecutionInfos.size()];
    int index = -1;
    for (BlockExecutionInfo blockInfo : blockExecutionInfos) {
      ++index;
      DataRefNodeFinder finder = new BTreeDataRefNodeFinder(blockInfo.getEachColumnValueSize());
      DataRefNode startDataBlock =
          finder.findFirstDataBlock(
              blockInfo.getDataBlock().getDataRefNode(), blockInfo.getStartKey());
      DataRefNode endDataBlock =
          finder.findLastDataBlock(
              blockInfo.getDataBlock().getDataRefNode(), blockInfo.getEndKey());

      this.totalNumberBlockletPerSlice[index] =
          endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1;
      totalNumberOfNode += this.totalNumberBlockletPerSlice[index];
      blockInfo.setFirstDataBlock(startDataBlock);
      blockInfo.setNumberOfBlockToScan(1);
    }
  }

  @Override
  public boolean hasNext() {
    return currentCounter < totalNumberOfNode;
  }

  protected int updateSliceIndexToBeExecuted() {
    Arrays.fill(blockIndexToBeExecuted, -1);
    int currentSliceIndex = 0;
    int i = 0;
    for (; i < (int) numberOfCores; ) {
      if (this.totalNumberBlockletPerSlice[currentSliceIndex]
          > this.numberOfBlockletExecutedPerBlock[currentSliceIndex]) {
        this.numberOfBlockletExecutedPerBlock[currentSliceIndex]++;
        blockIndexToBeExecuted[i] = currentSliceIndex;
        i++;
      }
      currentSliceIndex++;
      if (currentSliceIndex >= totalNumberBlockletPerSlice.length) {
        break;
      }
    }
    return i;
  }
}
/** It does not decode the dictionary. */
public class RawQueryResultPreparatorImpl
    extends AbstractQueryResultPreparator<List<ListBasedResultWrapper>, Object> {

  private static final LogService LOGGER =
      LogServiceFactory.getLogService(RawQueryResultPreparatorImpl.class.getName());

  private QuerySchemaInfo querySchemaInfo;

  public RawQueryResultPreparatorImpl(
      QueryExecutorProperties executerProperties, QueryModel queryModel) {
    super(executerProperties, queryModel);
    querySchemaInfo = new QuerySchemaInfo();
    querySchemaInfo.setKeyGenerator(queryExecuterProperties.keyStructureInfo.getKeyGenerator());
    querySchemaInfo.setMaskedByteIndexes(queryExecuterProperties.keyStructureInfo.getMaskedBytes());
    querySchemaInfo.setQueryDimensions(
        queryModel
            .getQueryDimension()
            .toArray(new QueryDimension[queryModel.getQueryDimension().size()]));
    querySchemaInfo.setQueryMeasures(
        queryModel
            .getQueryMeasures()
            .toArray(new QueryMeasure[queryModel.getQueryMeasures().size()]));
    int msrSize = queryExecuterProperties.measureDataTypes.length;
    int dimensionCount = queryModel.getQueryDimension().size();
    int[] queryOrder = new int[dimensionCount + msrSize];
    int[] queryReverseOrder = new int[dimensionCount + msrSize];
    for (int i = 0; i < dimensionCount; i++) {
      queryOrder[queryModel.getQueryDimension().get(i).getQueryOrder()] = i;
      queryReverseOrder[i] = queryModel.getQueryDimension().get(i).getQueryOrder();
    }
    for (int i = 0; i < msrSize; i++) {
      queryOrder[queryModel.getQueryMeasures().get(i).getQueryOrder()] = i + dimensionCount;
      queryReverseOrder[i + dimensionCount] = queryModel.getQueryMeasures().get(i).getQueryOrder();
    }
    querySchemaInfo.setQueryOrder(queryOrder);
    querySchemaInfo.setQueryReverseOrder(queryReverseOrder);
  }

  @Override
  public BatchResult prepareQueryResult(
      Result<List<ListBasedResultWrapper>, Object> scannedResult) {
    if ((null == scannedResult || scannedResult.size() < 1)) {
      return new BatchRawResult();
    }
    QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions();
    int msrSize = queryExecuterProperties.measureDataTypes.length;
    int dimSize = queryDimensions.length;
    int[] order = querySchemaInfo.getQueryReverseOrder();
    Object[][] resultData = new Object[scannedResult.size()][];
    Object[] value;
    Object[] row;
    int counter = 0;
    if (queryModel.isRawBytesDetailQuery()) {
      while (scannedResult.hasNext()) {
        value = scannedResult.getValue();
        row = new Object[msrSize + 1];
        row[0] = scannedResult.getKey();
        if (value != null) {
          assert (value.length == msrSize);
          System.arraycopy(value, 0, row, 1, msrSize);
        }
        resultData[counter] = row;
        counter++;
      }
    } else {
      while (scannedResult.hasNext()) {
        value = scannedResult.getValue();
        row = new Object[msrSize + dimSize];
        ByteArrayWrapper key = scannedResult.getKey();
        if (key != null) {
          long[] surrogateResult =
              querySchemaInfo
                  .getKeyGenerator()
                  .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
          int noDictionaryColumnIndex = 0;
          for (int i = 0; i < dimSize; i++) {
            if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
              row[order[i]] =
                  DataTypeUtil.getDataBasedOnDataType(
                      new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
                      queryDimensions[i].getDimension().getDataType());
            } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
              DirectDictionaryGenerator directDictionaryGenerator =
                  DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(
                      queryDimensions[i].getDimension().getDataType());
              if (directDictionaryGenerator != null) {
                row[order[i]] =
                    directDictionaryGenerator.getValueFromSurrogate(
                        (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
              }
            } else {
              row[order[i]] =
                  (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
            }
          }
        }
        for (int i = 0; i < msrSize; i++) {
          row[order[i + queryDimensions.length]] = value[i];
        }
        resultData[counter] = row;
        counter++;
      }
    }

    LOGGER.info("###########################---- Total Number of records" + scannedResult.size());
    BatchRawResult result = new BatchRawResult();
    result.setRows(resultData);
    return result;
  }
}