001/*
002 * Licensed to the Apache Software Foundation (ASF) under one or more
003 * contributor license agreements.  See the NOTICE file distributed with
004 * this work for additional information regarding copyright ownership.
005 * The ASF licenses this file to you under the Apache License, Version 2.0
006 * (the "License"); you may not use this file except in compliance with
007 * the License.  You may obtain a copy of the License at
008 *
009 * http://www.apache.org/licenses/LICENSE-2.0
010 *
011 * Unless required by applicable law or agreed to in writing, software
012 * distributed under the License is distributed on an "AS IS" BASIS,
013 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
014 * See the License for the specific language governing permissions and
015 * limitations under the License.
016 */
017package org.apache.hadoop.hbase.quotas;
018
019import java.io.IOException;
020import java.util.ArrayList;
021import java.util.Collection;
022import java.util.Collections;
023import java.util.HashMap;
024import java.util.HashSet;
025import java.util.Iterator;
026import java.util.List;
027import java.util.Map;
028import java.util.Map.Entry;
029import java.util.Objects;
030import java.util.Set;
031import java.util.concurrent.locks.ReentrantReadWriteLock;
032import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
033import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
034import java.util.function.Predicate;
035import java.util.stream.Collectors;
036
037import org.apache.commons.lang3.builder.HashCodeBuilder;
038import org.apache.hadoop.conf.Configuration;
039import org.apache.hadoop.fs.FileStatus;
040import org.apache.hadoop.fs.FileSystem;
041import org.apache.hadoop.fs.Path;
042import org.apache.hadoop.hbase.TableName;
043import org.apache.hadoop.hbase.client.Connection;
044import org.apache.hadoop.hbase.client.Get;
045import org.apache.hadoop.hbase.client.Put;
046import org.apache.hadoop.hbase.client.Result;
047import org.apache.hadoop.hbase.client.Table;
048import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
049import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
050import org.apache.hadoop.hbase.util.FSUtils;
051import org.apache.hadoop.hbase.util.HFileArchiveUtil;
052import org.apache.hadoop.util.StringUtils;
053import org.apache.yetus.audience.InterfaceAudience;
054
055import org.slf4j.Logger;
056import org.slf4j.LoggerFactory;
057
058import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap;
059import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
060import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
061
062import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
063import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
064import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
065import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles;
066import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile;
067
068/**
069 * Tracks file archiving and updates the hbase quota table.
070 */
071@InterfaceAudience.Private
072public class FileArchiverNotifierImpl implements FileArchiverNotifier {
073  private static final Logger LOG = LoggerFactory.getLogger(FileArchiverNotifierImpl.class);
074  private final Connection conn;
075  private final Configuration conf;
076  private final FileSystem fs;
077  private final TableName tn;
078  private final ReadLock readLock;
079  private final WriteLock writeLock;
080  private volatile long lastFullCompute = Long.MIN_VALUE;
081  private List<String> currentSnapshots = Collections.emptyList();
082  private static final Map<String,Object> NAMESPACE_LOCKS = new HashMap<>();
083
084  /**
085   * An Exception thrown when SnapshotSize updates to hbase:quota fail to be written.
086   */
087  @InterfaceAudience.Private
088  public static class QuotaSnapshotSizeSerializationException extends IOException {
089    private static final long serialVersionUID = 1L;
090
091    public QuotaSnapshotSizeSerializationException(String msg) {
092      super(msg);
093    }
094  }
095
096  public FileArchiverNotifierImpl(
097      Connection conn, Configuration conf, FileSystem fs, TableName tn) {
098    this.conn = conn;
099    this.conf = conf;
100    this.fs = fs;
101    this.tn = tn;
102    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
103    readLock = lock.readLock();
104    writeLock = lock.writeLock();
105  }
106
107  static synchronized Object getLockForNamespace(String namespace) {
108    return NAMESPACE_LOCKS.computeIfAbsent(namespace, (ns) -> new Object());
109  }
110
111  /**
112   * Returns a strictly-increasing measure of time extracted by {@link System#nanoTime()}.
113   */
114  long getLastFullCompute() {
115    return lastFullCompute;
116  }
117
118  @Override
119  public void addArchivedFiles(Set<Entry<String, Long>> fileSizes) throws IOException {
120    long start = System.nanoTime();
121    readLock.lock();
122    try {
123      // We want to catch the case where we got an archival request, but there was a full
124      // re-computation in progress that was blocking us. Most likely, the full computation is going
125      // to already include the changes we were going to make.
126      //
127      // Same as "start < lastFullCompute" but avoiding numeric overflow per the
128      // System.nanoTime() javadoc
129      if (lastFullCompute != Long.MIN_VALUE && start - lastFullCompute < 0) {
130        if (LOG.isTraceEnabled()) {
131          LOG.trace("A full computation was performed after this request was received."
132              + " Ignoring requested updates: " + fileSizes);
133        }
134        return;
135      }
136
137      if (LOG.isTraceEnabled()) {
138        LOG.trace("currentSnapshots: " + currentSnapshots + " fileSize: "+ fileSizes);
139      }
140
141      // Write increment to quota table for the correct snapshot. Only do this if we have snapshots
142      // and some files that were archived.
143      if (!currentSnapshots.isEmpty() && !fileSizes.isEmpty()) {
144        // We get back the files which no snapshot referenced (the files which will be deleted soon)
145        groupArchivedFiledBySnapshotAndRecordSize(currentSnapshots, fileSizes);
146      }
147    } finally {
148      readLock.unlock();
149    }
150  }
151
152  /**
153   * For each file in the map, this updates the first snapshot (lexicographic snapshot name) that
154   * references this file. The result of this computation is serialized to the quota table.
155   *
156   * @param snapshots A collection of HBase snapshots to group the files into
157   * @param fileSizes A map of file names to their sizes
158   */
159  void groupArchivedFiledBySnapshotAndRecordSize(
160      List<String> snapshots, Set<Entry<String, Long>> fileSizes) throws IOException {
161    // Make a copy as we'll modify it.
162    final Map<String,Long> filesToUpdate = new HashMap<>(fileSizes.size());
163    for (Entry<String,Long> entry : fileSizes) {
164      filesToUpdate.put(entry.getKey(), entry.getValue());
165    }
166    // Track the change in size to each snapshot
167    final Map<String,Long> snapshotSizeChanges = new HashMap<>();
168    for (String snapshot : snapshots) {
169      // For each file in `filesToUpdate`, check if `snapshot` refers to it.
170      // If `snapshot` does, remove it from `filesToUpdate` and add it to `snapshotSizeChanges`.
171      bucketFilesToSnapshot(snapshot, filesToUpdate, snapshotSizeChanges);
172      if (filesToUpdate.isEmpty()) {
173        // If we have no more files recently archived, we have nothing more to check
174        break;
175      }
176    }
177    // We have computed changes to the snapshot size, we need to record them.
178    if (!snapshotSizeChanges.isEmpty()) {
179      if (LOG.isTraceEnabled()) {
180        LOG.trace("Writing snapshot size changes for: " + snapshotSizeChanges);
181      }
182      persistSnapshotSizeChanges(snapshotSizeChanges);
183    }
184  }
185
186  /**
187   * For the given snapshot, find all files which this {@code snapshotName} references. After a file
188   * is found to be referenced by the snapshot, it is removed from {@code filesToUpdate} and
189   * {@code snapshotSizeChanges} is updated in concert.
190   *
191   * @param snapshotName The snapshot to check
192   * @param filesToUpdate A mapping of archived files to their size
193   * @param snapshotSizeChanges A mapping of snapshots and their change in size
194   */
195  void bucketFilesToSnapshot(
196      String snapshotName, Map<String,Long> filesToUpdate, Map<String,Long> snapshotSizeChanges)
197          throws IOException {
198    // A quick check to avoid doing work if the caller unnecessarily invoked this method.
199    if (filesToUpdate.isEmpty()) {
200      return;
201    }
202
203    Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(
204        snapshotName, FSUtils.getRootDir(conf));
205    SnapshotDescription sd = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
206    SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd);
207    // For each region referenced by the snapshot
208    for (SnapshotRegionManifest rm : manifest.getRegionManifests()) {
209      // For each column family in this region
210      for (FamilyFiles ff : rm.getFamilyFilesList()) {
211        // And each store file in that family
212        for (StoreFile sf : ff.getStoreFilesList()) {
213          Long valueOrNull = filesToUpdate.remove(sf.getName());
214          if (valueOrNull != null) {
215            // This storefile was recently archived, we should update this snapshot with its size
216            snapshotSizeChanges.merge(snapshotName, valueOrNull, Long::sum);
217          }
218          // Short-circuit, if we have no more files that were archived, we don't need to iterate
219          // over the rest of the snapshot.
220          if (filesToUpdate.isEmpty()) {
221            return;
222          }
223        }
224      }
225    }
226  }
227
228  /**
229   * Reads the current size for each snapshot to update, generates a new update based on that value,
230   * and then writes the new update.
231   *
232   * @param snapshotSizeChanges A map of snapshot name to size change
233   */
234  void persistSnapshotSizeChanges(Map<String,Long> snapshotSizeChanges) throws IOException {
235    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
236      // Create a list (with a more typical ordering implied)
237      final List<Entry<String,Long>> snapshotSizeEntries = new ArrayList<>(
238          snapshotSizeChanges.entrySet());
239      // Create the Gets for each snapshot we need to update
240      final List<Get> snapshotSizeGets = snapshotSizeEntries.stream()
241          .map((e) -> QuotaTableUtil.makeGetForSnapshotSize(tn, e.getKey()))
242          .collect(Collectors.toList());
243      final Iterator<Entry<String,Long>> iterator = snapshotSizeEntries.iterator();
244      // A List to store each Put we'll create from the Get's we retrieve
245      final List<Put> updates = new ArrayList<>(snapshotSizeEntries.size());
246
247      // TODO Push this down to the RegionServer with a coprocessor:
248      //
249      // We would really like to piggy-back on the row-lock already being grabbed
250      // to handle the update of the row in the quota table. However, because the value
251      // is a serialized protobuf, the standard Increment API doesn't work for us. With a CP, we
252      // can just send the size deltas to the RS and atomically update the serialized PB object
253      // while relying on the row-lock for synchronization.
254      //
255      // Synchronizing on the namespace string is a "minor smell" but passable as this is
256      // only invoked via a single caller (the active Master). Using the namespace name lets us
257      // have some parallelism without worry of on caller seeing stale data from the quota table.
258      synchronized (getLockForNamespace(tn.getNamespaceAsString())) {
259        final Result[] existingSnapshotSizes = quotaTable.get(snapshotSizeGets);
260        long totalSizeChange = 0;
261        // Read the current size values (if they exist) to generate the new value
262        for (Result result : existingSnapshotSizes) {
263          Entry<String,Long> entry = iterator.next();
264          String snapshot = entry.getKey();
265          Long size = entry.getValue();
266          // Track the total size change for the namespace this table belongs in
267          totalSizeChange += size;
268          // Get the size of the previous value (or zero)
269          long previousSize = getSnapshotSizeFromResult(result);
270          // Create an update. A file was archived from the table, so the table's size goes
271          // down, but the snapshot's size goes up.
272          updates.add(QuotaTableUtil.createPutForSnapshotSize(tn, snapshot, previousSize + size));
273        }
274
275        // Create an update for the summation of all snapshots in the namespace
276        if (totalSizeChange != 0) {
277          long previousSize = getPreviousNamespaceSnapshotSize(
278              quotaTable, tn.getNamespaceAsString());
279          updates.add(QuotaTableUtil.createPutForNamespaceSnapshotSize(
280              tn.getNamespaceAsString(), previousSize + totalSizeChange));
281        }
282
283        // Send all of the quota table updates in one batch.
284        List<Object> failures = new ArrayList<>();
285        final Object[] results = new Object[updates.size()];
286        quotaTable.batch(updates, results);
287        for (Object result : results) {
288          // A null result is an error condition (all RPC attempts failed)
289          if (!(result instanceof Result)) {
290            failures.add(result);
291          }
292        }
293        // Propagate a failure if any updates failed
294        if (!failures.isEmpty()) {
295          throw new QuotaSnapshotSizeSerializationException(
296              "Failed to write some snapshot size updates: " + failures);
297        }
298      }
299    } catch (InterruptedException e) {
300      Thread.currentThread().interrupt();
301      return;
302    }
303  }
304
305  /**
306   * Fetches the current size of all snapshots in the given {@code namespace}.
307   *
308   * @param quotaTable The HBase quota table
309   * @param namespace Namespace to fetch the sum of snapshot sizes for
310   * @return The size of all snapshot sizes for the namespace in bytes.
311   */
312  long getPreviousNamespaceSnapshotSize(Table quotaTable, String namespace) throws IOException {
313    // Update the size of each snapshot for all snapshots in a namespace.
314    Result r = quotaTable.get(
315        QuotaTableUtil.createGetNamespaceSnapshotSize(namespace));
316    return getSnapshotSizeFromResult(r);
317  }
318
319  /**
320   * Extracts the size component from a serialized {@link SpaceQuotaSnapshot} protobuf.
321   *
322   * @param r A Result containing one cell with a SpaceQuotaSnapshot protobuf
323   * @return The size in bytes of the snapshot.
324   */
325  long getSnapshotSizeFromResult(Result r) throws InvalidProtocolBufferException {
326    // Per javadoc, Result should only be null if an exception was thrown. So, if we're here,
327    // we should be non-null. If we can't advance to the first cell, same as "no cell".
328    if (!r.isEmpty() && r.advance()) {
329      return QuotaTableUtil.parseSnapshotSize(r.current());
330    }
331    return 0L;
332  }
333
334  @Override
335  public long computeAndStoreSnapshotSizes(
336      Collection<String> currentSnapshots) throws IOException {
337    // Record what the current snapshots are
338    this.currentSnapshots = new ArrayList<>(currentSnapshots);
339    Collections.sort(this.currentSnapshots);
340
341    // compute new size for table + snapshots for that table
342    List<SnapshotWithSize> snapshotSizes = computeSnapshotSizes(this.currentSnapshots);
343    if (LOG.isTraceEnabled()) {
344      LOG.trace("Computed snapshot sizes for " + tn + " of " + snapshotSizes);
345    }
346
347    // Compute the total size of all snapshots against our table
348    final long totalSnapshotSize = snapshotSizes.stream().mapToLong((sws) -> sws.getSize()).sum();
349
350    writeLock.lock();
351    try {
352      // Persist the size of each snapshot
353      try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
354        persistSnapshotSizes(quotaTable, snapshotSizes);
355      }
356
357      // Report the last time we did a recomputation
358      lastFullCompute = System.nanoTime();
359
360      return totalSnapshotSize;
361    } finally {
362      writeLock.unlock();
363    }
364  }
365
366  @Override
367  public String toString() {
368    StringBuilder sb = new StringBuilder();
369    sb.append(getClass().getSimpleName()).append("[");
370    sb.append("tableName=").append(tn).append(", currentSnapshots=");
371    sb.append(currentSnapshots).append(", lastFullCompute=").append(lastFullCompute);
372    return sb.append("]").toString();
373  }
374
375  /**
376   * Computes the size of each snapshot against the table referenced by {@code this}.
377   *
378   * @param snapshots A sorted list of snapshots against {@code tn}.
379   * @return A list of the size for each snapshot against {@code tn}.
380   */
381  List<SnapshotWithSize> computeSnapshotSizes(List<String> snapshots) throws IOException {
382    final List<SnapshotWithSize> snapshotSizes = new ArrayList<>(snapshots.size());
383    final Path rootDir = FSUtils.getRootDir(conf);
384
385    // Get the map of store file names to store file path for this table
386    final Set<String> tableReferencedStoreFiles;
387    try {
388      tableReferencedStoreFiles = FSUtils.getTableStoreFilePathMap(fs, rootDir).keySet();
389    } catch (InterruptedException e) {
390      Thread.currentThread().interrupt();
391      return null;
392    }
393
394    if (LOG.isTraceEnabled()) {
395      LOG.trace("Paths for " + tn + ": " + tableReferencedStoreFiles);
396    }
397
398    // For each snapshot on this table, get the files which the snapshot references which
399    // the table does not.
400    Set<String> snapshotReferencedFiles = new HashSet<>();
401    for (String snapshotName : snapshots) {
402      Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
403      SnapshotDescription sd = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
404      SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd);
405
406      if (LOG.isTraceEnabled()) {
407        LOG.trace("Files referenced by other snapshots: " + snapshotReferencedFiles);
408      }
409
410      // Get the set of files from the manifest that this snapshot references which are not also
411      // referenced by the originating table.
412      Set<StoreFileReference> unreferencedStoreFileNames = getStoreFilesFromSnapshot(
413          manifest, (sfn) -> !tableReferencedStoreFiles.contains(sfn)
414              && !snapshotReferencedFiles.contains(sfn));
415
416      if (LOG.isTraceEnabled()) {
417        LOG.trace("Snapshot " + snapshotName + " solely references the files: "
418            + unreferencedStoreFileNames);
419      }
420
421      // Compute the size of the store files for this snapshot
422      long size = getSizeOfStoreFiles(tn, unreferencedStoreFileNames);
423      if (LOG.isTraceEnabled()) {
424        LOG.trace("Computed size of " + snapshotName + " to be " + size);
425      }
426
427      // Persist this snapshot's size into the map
428      snapshotSizes.add(new SnapshotWithSize(snapshotName, size));
429
430      // Make sure that we don't double-count the same file
431      for (StoreFileReference ref : unreferencedStoreFileNames) {
432        for (String fileNames : ref.getFamilyToFilesMapping().values()) {
433          snapshotReferencedFiles.add(fileNames);
434        }
435      }
436    }
437
438    return snapshotSizes;
439  }
440
441  /**
442   * Computes the size of each store file in {@code storeFileNames}
443   */
444  long getSizeOfStoreFiles(TableName tn, Set<StoreFileReference> storeFileNames) {
445    return storeFileNames.stream()
446        .collect(Collectors.summingLong((sfr) -> getSizeOfStoreFile(tn, sfr)));
447  }
448
449  /**
450   * Computes the size of the store files for a single region.
451   */
452  long getSizeOfStoreFile(TableName tn, StoreFileReference storeFileName) {
453    String regionName = storeFileName.getRegionName();
454    return storeFileName.getFamilyToFilesMapping()
455        .entries().stream()
456        .collect(Collectors.summingLong((e) ->
457            getSizeOfStoreFile(tn, regionName, e.getKey(), e.getValue())));
458  }
459
460  /**
461   * Computes the size of the store file given its name, region and family name in
462   * the archive directory.
463   */
464  long getSizeOfStoreFile(
465      TableName tn, String regionName, String family, String storeFile) {
466    Path familyArchivePath;
467    try {
468      familyArchivePath = HFileArchiveUtil.getStoreArchivePath(conf, tn, regionName, family);
469    } catch (IOException e) {
470      LOG.warn("Could not compute path for the archive directory for the region", e);
471      return 0L;
472    }
473    Path fileArchivePath = new Path(familyArchivePath, storeFile);
474    try {
475      if (fs.exists(fileArchivePath)) {
476        FileStatus[] status = fs.listStatus(fileArchivePath);
477        if (1 != status.length) {
478          LOG.warn("Expected " + fileArchivePath +
479              " to be a file but was a directory, ignoring reference");
480          return 0L;
481        }
482        return status[0].getLen();
483      }
484    } catch (IOException e) {
485      LOG.warn("Could not obtain the status of " + fileArchivePath, e);
486      return 0L;
487    }
488    LOG.warn("Expected " + fileArchivePath + " to exist but does not, ignoring reference.");
489    return 0L;
490  }
491
492  /**
493   * Extracts the names of the store files referenced by this snapshot which satisfy the given
494   * predicate (the predicate returns {@code true}).
495   */
496  Set<StoreFileReference> getStoreFilesFromSnapshot(
497      SnapshotManifest manifest, Predicate<String> filter) {
498    Set<StoreFileReference> references = new HashSet<>();
499    // For each region referenced by the snapshot
500    for (SnapshotRegionManifest rm : manifest.getRegionManifests()) {
501      StoreFileReference regionReference = new StoreFileReference(
502          ProtobufUtil.toRegionInfo(rm.getRegionInfo()).getEncodedName());
503
504      // For each column family in this region
505      for (FamilyFiles ff : rm.getFamilyFilesList()) {
506        final String familyName = ff.getFamilyName().toStringUtf8();
507        // And each store file in that family
508        for (StoreFile sf : ff.getStoreFilesList()) {
509          String storeFileName = sf.getName();
510          // A snapshot only "inherits" a files size if it uniquely refers to it (no table
511          // and no other snapshot references it).
512          if (filter.test(storeFileName)) {
513            regionReference.addFamilyStoreFile(familyName, storeFileName);
514          }
515        }
516      }
517      // Only add this Region reference if we retained any files.
518      if (!regionReference.getFamilyToFilesMapping().isEmpty()) {
519        references.add(regionReference);
520      }
521    }
522    return references;
523  }
524
525  /**
526   * Writes the snapshot sizes to the provided {@code table}.
527   */
528  void persistSnapshotSizes(
529      Table table, List<SnapshotWithSize> snapshotSizes) throws IOException {
530    // Convert each entry in the map to a Put and write them to the quota table
531    table.put(snapshotSizes
532        .stream()
533        .map(sws -> QuotaTableUtil.createPutForSnapshotSize(
534            tn, sws.getName(), sws.getSize()))
535        .collect(Collectors.toList()));
536  }
537
538  /**
539   * A struct encapsulating the name of a snapshot and its "size" on the filesystem. This size is
540   * defined as the amount of filesystem space taken by the files the snapshot refers to which
541   * the originating table no longer refers to.
542   */
543  static class SnapshotWithSize {
544    private final String name;
545    private final long size;
546
547    SnapshotWithSize(String name, long size) {
548      this.name = Objects.requireNonNull(name);
549      this.size = size;
550    }
551
552    String getName() {
553      return name;
554    }
555
556    long getSize() {
557      return size;
558    }
559
560    @Override
561    public int hashCode() {
562      return new HashCodeBuilder().append(name).append(size).toHashCode();
563    }
564
565    @Override
566    public boolean equals(Object o) {
567      if (this == o) {
568        return true;
569      }
570
571      if (!(o instanceof SnapshotWithSize)) {
572        return false;
573      }
574
575      SnapshotWithSize other = (SnapshotWithSize) o;
576      return name.equals(other.name) && size == other.size;
577    }
578
579    @Override
580    public String toString() {
581      StringBuilder sb = new StringBuilder(32);
582      return sb.append("SnapshotWithSize:[").append(name).append(" ")
583          .append(StringUtils.byteDesc(size)).append("]").toString();
584    }
585  }
586
587  /**
588   * A reference to a collection of files in the archive directory for a single region.
589   */
590  static class StoreFileReference {
591    private final String regionName;
592    private final Multimap<String,String> familyToFiles;
593
594    StoreFileReference(String regionName) {
595      this.regionName = Objects.requireNonNull(regionName);
596      familyToFiles = HashMultimap.create();
597    }
598
599    String getRegionName() {
600      return regionName;
601    }
602
603    Multimap<String,String> getFamilyToFilesMapping() {
604      return familyToFiles;
605    }
606
607    void addFamilyStoreFile(String family, String storeFileName) {
608      familyToFiles.put(family, storeFileName);
609    }
610
611    @Override
612    public int hashCode() {
613      return new HashCodeBuilder().append(regionName).append(familyToFiles).toHashCode();
614    }
615
616    @Override
617    public boolean equals(Object o) {
618      if (this == o) {
619        return true;
620      }
621      if (!(o instanceof StoreFileReference)) {
622        return false;
623      }
624      StoreFileReference other = (StoreFileReference) o;
625      return regionName.equals(other.regionName) && familyToFiles.equals(other.familyToFiles);
626    }
627
628    @Override
629    public String toString() {
630      StringBuilder sb = new StringBuilder();
631      return sb.append("StoreFileReference[region=").append(regionName).append(", files=")
632          .append(familyToFiles).append("]").toString();
633    }
634  }
635}