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;
019
020import java.io.BufferedInputStream;
021import java.io.DataInput;
022import java.io.DataInputStream;
023import java.io.IOException;
024import java.io.InputStream;
025import java.util.Arrays;
026import org.apache.commons.io.IOUtils;
027import org.apache.hadoop.fs.FSDataOutputStream;
028import org.apache.hadoop.fs.FileSystem;
029import org.apache.hadoop.fs.Path;
030import org.apache.hadoop.hbase.KeyValueUtil;
031import org.apache.hadoop.hbase.util.Bytes;
032import org.apache.yetus.audience.InterfaceAudience;
033
034import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
035
036import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
037import org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos;
038
039/**
040 * A reference to the top or bottom half of a store file where 'bottom' is the first half of the
041 * file containing the keys that sort lowest and 'top' is the second half of the file with keys that
042 * sort greater than those of the bottom half. The file referenced lives under a different region.
043 * References are made at region split time.
044 * <p>
045 * References work with a special half store file type. References know how to write out the
046 * reference format in the file system and are what is juggled when references are mixed in with
047 * direct store files. The half store file type is used reading the referred to file.
048 * <p>
049 * References to store files located over in some other region look like this in the file system
050 * <code>1278437856009925445.3323223323</code>: i.e. an id followed by hash of the referenced
051 * region. Note, a region is itself not splittable if it has instances of store file references.
052 * References are cleaned up by compactions.
053 */
054@InterfaceAudience.Private
055public class Reference {
056  private byte[] splitkey;
057  private Range region;
058
059  /**
060   * For split HStoreFiles, it specifies if the file covers the lower half or the upper half of the
061   * key range
062   */
063  static enum Range {
064    /** HStoreFile contains upper half of key range */
065    top,
066    /** HStoreFile contains lower half of key range */
067    bottom
068  }
069
070  /**
071   * n * @return A {@link Reference} that points at top half of a an hfile
072   */
073  public static Reference createTopReference(final byte[] splitRow) {
074    return new Reference(splitRow, Range.top);
075  }
076
077  /**
078   * n * @return A {@link Reference} that points at the bottom half of a an hfile
079   */
080  public static Reference createBottomReference(final byte[] splitRow) {
081    return new Reference(splitRow, Range.bottom);
082  }
083
084  /**
085   * Constructor
086   * @param splitRow This is row we are splitting around. n
087   */
088  Reference(final byte[] splitRow, final Range fr) {
089    this.splitkey = splitRow == null ? null : KeyValueUtil.createFirstOnRow(splitRow).getKey();
090    this.region = fr;
091  }
092
093  /**
094   * Used by serializations.
095   * @deprecated need by pb serialization
096   */
097  @Deprecated
098  // Make this private when it comes time to let go of this constructor.
099  // Needed by pb serialization.
100  public Reference() {
101    this(null, Range.bottom);
102  }
103
104  /**
105   * n
106   */
107  public Range getFileRegion() {
108    return this.region;
109  }
110
111  /**
112   * n
113   */
114  public byte[] getSplitKey() {
115    return splitkey;
116  }
117
118  /**
119   * @see java.lang.Object#toString()
120   */
121  @Override
122  public String toString() {
123    return "" + this.region;
124  }
125
126  public static boolean isTopFileRegion(final Range r) {
127    return r.equals(Range.top);
128  }
129
130  /**
131   * @deprecated Writables are going away. Use the pb serialization methods instead. Remove in a
132   *             release after 0.96 goes out. This is here only to migrate old Reference files
133   *             written with Writables before 0.96.
134   */
135  @Deprecated
136  public void readFields(DataInput in) throws IOException {
137    boolean tmp = in.readBoolean();
138    // If true, set region to top.
139    this.region = tmp ? Range.top : Range.bottom;
140    this.splitkey = Bytes.readByteArray(in);
141  }
142
143  public Path write(final FileSystem fs, final Path p) throws IOException {
144    FSDataOutputStream out = fs.create(p, false);
145    try {
146      out.write(toByteArray());
147    } finally {
148      out.close();
149    }
150    return p;
151  }
152
153  /**
154   * Read a Reference from FileSystem. nn * @return New Reference made from passed <code>p</code> n
155   */
156  public static Reference read(final FileSystem fs, final Path p) throws IOException {
157    InputStream in = fs.open(p);
158    try {
159      // I need to be able to move back in the stream if this is not a pb serialization so I can
160      // do the Writable decoding instead.
161      in = in.markSupported() ? in : new BufferedInputStream(in);
162      int pblen = ProtobufUtil.lengthOfPBMagic();
163      in.mark(pblen);
164      byte[] pbuf = new byte[pblen];
165      IOUtils.readFully(in, pbuf, 0, pblen);
166      // WATCHOUT! Return in middle of function!!!
167      if (ProtobufUtil.isPBMagicPrefix(pbuf)) return convert(FSProtos.Reference.parseFrom(in));
168      // Else presume Writables. Need to reset the stream since it didn't start w/ pb.
169      // We won't bother rewriting thie Reference as a pb since Reference is transitory.
170      in.reset();
171      Reference r = new Reference();
172      DataInputStream dis = new DataInputStream(in);
173      // Set in = dis so it gets the close below in the finally on our way out.
174      in = dis;
175      r.readFields(dis);
176      return r;
177    } finally {
178      in.close();
179    }
180  }
181
182  public FSProtos.Reference convert() {
183    FSProtos.Reference.Builder builder = FSProtos.Reference.newBuilder();
184    builder.setRange(isTopFileRegion(getFileRegion())
185      ? FSProtos.Reference.Range.TOP
186      : FSProtos.Reference.Range.BOTTOM);
187    builder.setSplitkey(UnsafeByteOperations.unsafeWrap(getSplitKey()));
188    return builder.build();
189  }
190
191  public static Reference convert(final FSProtos.Reference r) {
192    Reference result = new Reference();
193    result.splitkey = r.getSplitkey().toByteArray();
194    result.region = r.getRange() == FSProtos.Reference.Range.TOP ? Range.top : Range.bottom;
195    return result;
196  }
197
198  /**
199   * Use this when writing to a stream and you want to use the pb mergeDelimitedFrom (w/o the
200   * delimiter, pb reads to EOF which may not be what you want).
201   * @return This instance serialized as a delimited protobuf w/ a magic pb prefix. n
202   */
203  byte[] toByteArray() throws IOException {
204    return ProtobufUtil.prependPBMagic(convert().toByteArray());
205  }
206
207  @Override
208  public int hashCode() {
209    return Arrays.hashCode(splitkey) + region.hashCode();
210  }
211
212  @Override
213  public boolean equals(Object o) {
214    if (this == o) return true;
215    if (o == null) return false;
216    if (!(o instanceof Reference)) return false;
217
218    Reference r = (Reference) o;
219    if (splitkey != null && r.splitkey == null) return false;
220    if (splitkey == null && r.splitkey != null) return false;
221    if (splitkey != null && !Arrays.equals(splitkey, r.splitkey)) return false;
222
223    return region.equals(r.region);
224  }
225}