001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io.encoding;
019
020import java.io.DataOutputStream;
021import java.io.IOException;
022import org.apache.hadoop.hbase.Cell;
023import org.apache.hadoop.hbase.KeyValueUtil;
024import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
025import org.apache.hadoop.hbase.util.Bytes;
026import org.apache.yetus.audience.InterfaceAudience;
027import org.slf4j.Logger;
028import org.slf4j.LoggerFactory;
029
030@InterfaceAudience.Private
031public class RowIndexEncoderV1 {
032  private static final Logger LOG = LoggerFactory.getLogger(RowIndexEncoderV1.class);
033
034  /** The Cell previously appended. */
035  private Cell lastCell = null;
036
037  private DataOutputStream out;
038  private NoneEncoder encoder;
039  private int startOffset = -1;
040  private ByteArrayOutputStream rowsOffsetBAOS = new ByteArrayOutputStream(64 * 4);
041  private final HFileBlockEncodingContext context;
042
043  public RowIndexEncoderV1(DataOutputStream out, HFileBlockDefaultEncodingContext encodingCtx) {
044    this.out = out;
045    this.encoder = new NoneEncoder(out, encodingCtx);
046    this.context = encodingCtx;
047  }
048
049  public void write(Cell cell) throws IOException {
050    // checkRow uses comparator to check we are writing in order.
051    int extraBytesForRowIndex = 0;
052
053    if (!checkRow(cell)) {
054      if (startOffset < 0) {
055        startOffset = out.size();
056      }
057      rowsOffsetBAOS.writeInt(out.size() - startOffset);
058      // added for the int written in the previous line
059      extraBytesForRowIndex = Bytes.SIZEOF_INT;
060    }
061    lastCell = cell;
062    int size = encoder.write(cell);
063    context.getEncodingState().postCellEncode(size, size + extraBytesForRowIndex);
064  }
065
066  protected boolean checkRow(final Cell cell) throws IOException {
067    boolean isDuplicateRow = false;
068    if (cell == null) {
069      throw new IOException("Key cannot be null or empty");
070    }
071    if (lastCell != null) {
072      int keyComp = this.context.getHFileContext().getCellComparator().compareRows(lastCell, cell);
073      if (keyComp > 0) {
074        throw new IOException("Added a key not lexically larger than" + " previous. Current cell = "
075          + cell + ", lastCell = " + lastCell);
076      } else if (keyComp == 0) {
077        isDuplicateRow = true;
078      }
079    }
080    return isDuplicateRow;
081  }
082
083  public void flush() throws IOException {
084    int onDiskDataSize = 0;
085    if (startOffset >= 0) {
086      onDiskDataSize = out.size() - startOffset;
087    }
088    out.writeInt(rowsOffsetBAOS.size() / 4);
089    if (rowsOffsetBAOS.size() > 0) {
090      out.write(rowsOffsetBAOS.getBuffer(), 0, rowsOffsetBAOS.size());
091    }
092    out.writeInt(onDiskDataSize);
093    if (LOG.isTraceEnabled()) {
094      LOG.trace("RowNumber: " + rowsOffsetBAOS.size() / 4 + ", onDiskDataSize: " + onDiskDataSize
095        + ", totalOnDiskSize: " + (out.size() - startOffset));
096    }
097  }
098
099  void beforeShipped() {
100    if (this.lastCell != null) {
101      this.lastCell = KeyValueUtil.toNewKeyCell(this.lastCell);
102    }
103  }
104}