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.quotas;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.Collection;
023import java.util.Collections;
024import java.util.HashMap;
025import java.util.HashSet;
026import java.util.Iterator;
027import java.util.List;
028import java.util.Map;
029import java.util.Map.Entry;
030import java.util.Objects;
031import java.util.Set;
032import java.util.concurrent.locks.ReentrantReadWriteLock;
033import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
034import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
035import java.util.function.Predicate;
036import java.util.stream.Collectors;
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.CommonFSUtils;
051import org.apache.hadoop.hbase.util.FSUtils;
052import org.apache.hadoop.hbase.util.HFileArchiveUtil;
053import org.apache.hadoop.util.StringUtils;
054import org.apache.yetus.audience.InterfaceAudience;
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(Connection conn, Configuration conf, FileSystem fs,
097    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   * @param snapshots A collection of HBase snapshots to group the files into
156   * @param fileSizes A map of file names to their sizes
157   */
158  void groupArchivedFiledBySnapshotAndRecordSize(List<String> snapshots,
159    Set<Entry<String, Long>> fileSizes) throws IOException {
160    // Make a copy as we'll modify it.
161    final Map<String, Long> filesToUpdate = new HashMap<>(fileSizes.size());
162    for (Entry<String, Long> entry : fileSizes) {
163      filesToUpdate.put(entry.getKey(), entry.getValue());
164    }
165    // Track the change in size to each snapshot
166    final Map<String, Long> snapshotSizeChanges = new HashMap<>();
167    for (String snapshot : snapshots) {
168      // For each file in `filesToUpdate`, check if `snapshot` refers to it.
169      // If `snapshot` does, remove it from `filesToUpdate` and add it to `snapshotSizeChanges`.
170      bucketFilesToSnapshot(snapshot, filesToUpdate, snapshotSizeChanges);
171      if (filesToUpdate.isEmpty()) {
172        // If we have no more files recently archived, we have nothing more to check
173        break;
174      }
175    }
176    // We have computed changes to the snapshot size, we need to record them.
177    if (!snapshotSizeChanges.isEmpty()) {
178      if (LOG.isTraceEnabled()) {
179        LOG.trace("Writing snapshot size changes for: " + snapshotSizeChanges);
180      }
181      persistSnapshotSizeChanges(snapshotSizeChanges);
182    }
183  }
184
185  /**
186   * For the given snapshot, find all files which this {@code snapshotName} references. After a file
187   * is found to be referenced by the snapshot, it is removed from {@code filesToUpdate} and
188   * {@code snapshotSizeChanges} is updated in concert.
189   * @param snapshotName        The snapshot to check
190   * @param filesToUpdate       A mapping of archived files to their size
191   * @param snapshotSizeChanges A mapping of snapshots and their change in size
192   */
193  void bucketFilesToSnapshot(String snapshotName, Map<String, Long> filesToUpdate,
194    Map<String, Long> snapshotSizeChanges) throws IOException {
195    // A quick check to avoid doing work if the caller unnecessarily invoked this method.
196    if (filesToUpdate.isEmpty()) {
197      return;
198    }
199
200    Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName,
201      CommonFSUtils.getRootDir(conf));
202    SnapshotDescription sd = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
203    SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd);
204    // For each region referenced by the snapshot
205    for (SnapshotRegionManifest rm : manifest.getRegionManifests()) {
206      // For each column family in this region
207      for (FamilyFiles ff : rm.getFamilyFilesList()) {
208        // And each store file in that family
209        for (StoreFile sf : ff.getStoreFilesList()) {
210          Long valueOrNull = filesToUpdate.remove(sf.getName());
211          if (valueOrNull != null) {
212            // This storefile was recently archived, we should update this snapshot with its size
213            snapshotSizeChanges.merge(snapshotName, valueOrNull, Long::sum);
214          }
215          // Short-circuit, if we have no more files that were archived, we don't need to iterate
216          // over the rest of the snapshot.
217          if (filesToUpdate.isEmpty()) {
218            return;
219          }
220        }
221      }
222    }
223  }
224
225  /**
226   * Reads the current size for each snapshot to update, generates a new update based on that value,
227   * and then writes the new update.
228   * @param snapshotSizeChanges A map of snapshot name to size change
229   */
230  void persistSnapshotSizeChanges(Map<String, Long> snapshotSizeChanges) throws IOException {
231    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
232      // Create a list (with a more typical ordering implied)
233      final List<Entry<String, Long>> snapshotSizeEntries =
234        new ArrayList<>(snapshotSizeChanges.entrySet());
235      // Create the Gets for each snapshot we need to update
236      final List<Get> snapshotSizeGets = snapshotSizeEntries.stream()
237        .map((e) -> QuotaTableUtil.makeGetForSnapshotSize(tn, e.getKey()))
238        .collect(Collectors.toList());
239      final Iterator<Entry<String, Long>> iterator = snapshotSizeEntries.iterator();
240      // A List to store each Put we'll create from the Get's we retrieve
241      final List<Put> updates = new ArrayList<>(snapshotSizeEntries.size());
242
243      // TODO Push this down to the RegionServer with a coprocessor:
244      //
245      // We would really like to piggy-back on the row-lock already being grabbed
246      // to handle the update of the row in the quota table. However, because the value
247      // is a serialized protobuf, the standard Increment API doesn't work for us. With a CP, we
248      // can just send the size deltas to the RS and atomically update the serialized PB object
249      // while relying on the row-lock for synchronization.
250      //
251      // Synchronizing on the namespace string is a "minor smell" but passable as this is
252      // only invoked via a single caller (the active Master). Using the namespace name lets us
253      // have some parallelism without worry of on caller seeing stale data from the quota table.
254      synchronized (getLockForNamespace(tn.getNamespaceAsString())) {
255        final Result[] existingSnapshotSizes = quotaTable.get(snapshotSizeGets);
256        long totalSizeChange = 0;
257        // Read the current size values (if they exist) to generate the new value
258        for (Result result : existingSnapshotSizes) {
259          Entry<String, Long> entry = iterator.next();
260          String snapshot = entry.getKey();
261          Long size = entry.getValue();
262          // Track the total size change for the namespace this table belongs in
263          totalSizeChange += size;
264          // Get the size of the previous value (or zero)
265          long previousSize = getSnapshotSizeFromResult(result);
266          // Create an update. A file was archived from the table, so the table's size goes
267          // down, but the snapshot's size goes up.
268          updates.add(QuotaTableUtil.createPutForSnapshotSize(tn, snapshot, previousSize + size));
269        }
270
271        // Create an update for the summation of all snapshots in the namespace
272        if (totalSizeChange != 0) {
273          long previousSize =
274            getPreviousNamespaceSnapshotSize(quotaTable, tn.getNamespaceAsString());
275          updates.add(QuotaTableUtil.createPutForNamespaceSnapshotSize(tn.getNamespaceAsString(),
276            previousSize + totalSizeChange));
277        }
278
279        // Send all of the quota table updates in one batch.
280        List<Object> failures = new ArrayList<>();
281        final Object[] results = new Object[updates.size()];
282        quotaTable.batch(updates, results);
283        for (Object result : results) {
284          // A null result is an error condition (all RPC attempts failed)
285          if (!(result instanceof Result)) {
286            failures.add(result);
287          }
288        }
289        // Propagate a failure if any updates failed
290        if (!failures.isEmpty()) {
291          throw new QuotaSnapshotSizeSerializationException(
292            "Failed to write some snapshot size updates: " + failures);
293        }
294      }
295    } catch (InterruptedException e) {
296      Thread.currentThread().interrupt();
297      return;
298    }
299  }
300
301  /**
302   * Fetches the current size of all snapshots in the given {@code namespace}.
303   * @param quotaTable The HBase quota table
304   * @param namespace  Namespace to fetch the sum of snapshot sizes for
305   * @return The size of all snapshot sizes for the namespace in bytes.
306   */
307  long getPreviousNamespaceSnapshotSize(Table quotaTable, String namespace) throws IOException {
308    // Update the size of each snapshot for all snapshots in a namespace.
309    Result r = quotaTable.get(QuotaTableUtil.createGetNamespaceSnapshotSize(namespace));
310    return getSnapshotSizeFromResult(r);
311  }
312
313  /**
314   * Extracts the size component from a serialized {@link SpaceQuotaSnapshot} protobuf.
315   * @param r A Result containing one cell with a SpaceQuotaSnapshot protobuf
316   * @return The size in bytes of the snapshot.
317   */
318  long getSnapshotSizeFromResult(Result r) throws InvalidProtocolBufferException {
319    // Per javadoc, Result should only be null if an exception was thrown. So, if we're here,
320    // we should be non-null. If we can't advance to the first cell, same as "no cell".
321    if (!r.isEmpty() && r.advance()) {
322      return QuotaTableUtil.parseSnapshotSize(r.current());
323    }
324    return 0L;
325  }
326
327  @Override
328  public long computeAndStoreSnapshotSizes(Collection<String> currentSnapshots) throws IOException {
329    // Record what the current snapshots are
330    this.currentSnapshots = new ArrayList<>(currentSnapshots);
331    Collections.sort(this.currentSnapshots);
332
333    // compute new size for table + snapshots for that table
334    List<SnapshotWithSize> snapshotSizes = computeSnapshotSizes(this.currentSnapshots);
335    if (LOG.isTraceEnabled()) {
336      LOG.trace("Computed snapshot sizes for " + tn + " of " + snapshotSizes);
337    }
338
339    // Compute the total size of all snapshots against our table
340    final long totalSnapshotSize = snapshotSizes.stream().mapToLong((sws) -> sws.getSize()).sum();
341
342    writeLock.lock();
343    try {
344      // Persist the size of each snapshot
345      try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
346        persistSnapshotSizes(quotaTable, snapshotSizes);
347      }
348
349      // Report the last time we did a recomputation
350      lastFullCompute = System.nanoTime();
351
352      return totalSnapshotSize;
353    } finally {
354      writeLock.unlock();
355    }
356  }
357
358  @Override
359  public String toString() {
360    StringBuilder sb = new StringBuilder();
361    sb.append(getClass().getSimpleName()).append("[");
362    sb.append("tableName=").append(tn).append(", currentSnapshots=");
363    sb.append(currentSnapshots).append(", lastFullCompute=").append(lastFullCompute);
364    return sb.append("]").toString();
365  }
366
367  /**
368   * Computes the size of each snapshot against the table referenced by {@code this}.
369   * @param snapshots A sorted list of snapshots against {@code tn}.
370   * @return A list of the size for each snapshot against {@code tn}.
371   */
372  List<SnapshotWithSize> computeSnapshotSizes(List<String> snapshots) throws IOException {
373    final List<SnapshotWithSize> snapshotSizes = new ArrayList<>(snapshots.size());
374    final Path rootDir = CommonFSUtils.getRootDir(conf);
375
376    // Get the map of store file names to store file path for this table
377    final Set<String> tableReferencedStoreFiles;
378    try {
379      tableReferencedStoreFiles = FSUtils.getTableStoreFilePathMap(fs, rootDir).keySet();
380    } catch (InterruptedException e) {
381      Thread.currentThread().interrupt();
382      return null;
383    }
384
385    if (LOG.isTraceEnabled()) {
386      LOG.trace("Paths for " + tn + ": " + tableReferencedStoreFiles);
387    }
388
389    // For each snapshot on this table, get the files which the snapshot references which
390    // the table does not.
391    Set<String> snapshotReferencedFiles = new HashSet<>();
392    for (String snapshotName : snapshots) {
393      Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
394      SnapshotDescription sd = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
395      SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd);
396
397      if (LOG.isTraceEnabled()) {
398        LOG.trace("Files referenced by other snapshots: " + snapshotReferencedFiles);
399      }
400
401      // Get the set of files from the manifest that this snapshot references which are not also
402      // referenced by the originating table.
403      Set<StoreFileReference> unreferencedStoreFileNames =
404        getStoreFilesFromSnapshot(manifest, (sfn) -> !tableReferencedStoreFiles.contains(sfn)
405          && !snapshotReferencedFiles.contains(sfn));
406
407      if (LOG.isTraceEnabled()) {
408        LOG.trace("Snapshot " + snapshotName + " solely references the files: "
409          + unreferencedStoreFileNames);
410      }
411
412      // Compute the size of the store files for this snapshot
413      long size = getSizeOfStoreFiles(tn, unreferencedStoreFileNames);
414      if (LOG.isTraceEnabled()) {
415        LOG.trace("Computed size of " + snapshotName + " to be " + size);
416      }
417
418      // Persist this snapshot's size into the map
419      snapshotSizes.add(new SnapshotWithSize(snapshotName, size));
420
421      // Make sure that we don't double-count the same file
422      for (StoreFileReference ref : unreferencedStoreFileNames) {
423        for (String fileNames : ref.getFamilyToFilesMapping().values()) {
424          snapshotReferencedFiles.add(fileNames);
425        }
426      }
427    }
428
429    return snapshotSizes;
430  }
431
432  /**
433   * Computes the size of each store file in {@code storeFileNames}
434   */
435  long getSizeOfStoreFiles(TableName tn, Set<StoreFileReference> storeFileNames) {
436    return storeFileNames.stream()
437      .collect(Collectors.summingLong((sfr) -> getSizeOfStoreFile(tn, sfr)));
438  }
439
440  /**
441   * Computes the size of the store files for a single region.
442   */
443  long getSizeOfStoreFile(TableName tn, StoreFileReference storeFileName) {
444    String regionName = storeFileName.getRegionName();
445    return storeFileName.getFamilyToFilesMapping().entries().stream().collect(
446      Collectors.summingLong((e) -> getSizeOfStoreFile(tn, regionName, e.getKey(), e.getValue())));
447  }
448
449  /**
450   * Computes the size of the store file given its name, region and family name in the archive
451   * directory.
452   */
453  long getSizeOfStoreFile(TableName tn, String regionName, String family, String storeFile) {
454    Path familyArchivePath;
455    try {
456      familyArchivePath = HFileArchiveUtil.getStoreArchivePath(conf, tn, regionName, family);
457    } catch (IOException e) {
458      LOG.warn("Could not compute path for the archive directory for the region", e);
459      return 0L;
460    }
461    Path fileArchivePath = new Path(familyArchivePath, storeFile);
462    try {
463      if (fs.exists(fileArchivePath)) {
464        FileStatus[] status = fs.listStatus(fileArchivePath);
465        if (1 != status.length) {
466          LOG.warn("Expected " + fileArchivePath
467            + " to be a file but was a directory, ignoring reference");
468          return 0L;
469        }
470        return status[0].getLen();
471      }
472    } catch (IOException e) {
473      LOG.warn("Could not obtain the status of " + fileArchivePath, e);
474      return 0L;
475    }
476    LOG.warn("Expected " + fileArchivePath + " to exist but does not, ignoring reference.");
477    return 0L;
478  }
479
480  /**
481   * Extracts the names of the store files referenced by this snapshot which satisfy the given
482   * predicate (the predicate returns {@code true}).
483   */
484  Set<StoreFileReference> getStoreFilesFromSnapshot(SnapshotManifest manifest,
485    Predicate<String> filter) {
486    Set<StoreFileReference> references = new HashSet<>();
487    // For each region referenced by the snapshot
488    for (SnapshotRegionManifest rm : manifest.getRegionManifests()) {
489      StoreFileReference regionReference =
490        new StoreFileReference(ProtobufUtil.toRegionInfo(rm.getRegionInfo()).getEncodedName());
491
492      // For each column family in this region
493      for (FamilyFiles ff : rm.getFamilyFilesList()) {
494        final String familyName = ff.getFamilyName().toStringUtf8();
495        // And each store file in that family
496        for (StoreFile sf : ff.getStoreFilesList()) {
497          String storeFileName = sf.getName();
498          // A snapshot only "inherits" a files size if it uniquely refers to it (no table
499          // and no other snapshot references it).
500          if (filter.test(storeFileName)) {
501            regionReference.addFamilyStoreFile(familyName, storeFileName);
502          }
503        }
504      }
505      // Only add this Region reference if we retained any files.
506      if (!regionReference.getFamilyToFilesMapping().isEmpty()) {
507        references.add(regionReference);
508      }
509    }
510    return references;
511  }
512
513  /**
514   * Writes the snapshot sizes to the provided {@code table}.
515   */
516  void persistSnapshotSizes(Table table, List<SnapshotWithSize> snapshotSizes) throws IOException {
517    // Convert each entry in the map to a Put and write them to the quota table
518    table.put(snapshotSizes.stream()
519      .map(sws -> QuotaTableUtil.createPutForSnapshotSize(tn, sws.getName(), sws.getSize()))
520      .collect(Collectors.toList()));
521  }
522
523  /**
524   * A struct encapsulating the name of a snapshot and its "size" on the filesystem. This size is
525   * defined as the amount of filesystem space taken by the files the snapshot refers to which the
526   * originating table no longer refers to.
527   */
528  static class SnapshotWithSize {
529    private final String name;
530    private final long size;
531
532    SnapshotWithSize(String name, long size) {
533      this.name = Objects.requireNonNull(name);
534      this.size = size;
535    }
536
537    String getName() {
538      return name;
539    }
540
541    long getSize() {
542      return size;
543    }
544
545    @Override
546    public int hashCode() {
547      return new HashCodeBuilder().append(name).append(size).toHashCode();
548    }
549
550    @Override
551    public boolean equals(Object o) {
552      if (this == o) {
553        return true;
554      }
555
556      if (!(o instanceof SnapshotWithSize)) {
557        return false;
558      }
559
560      SnapshotWithSize other = (SnapshotWithSize) o;
561      return name.equals(other.name) && size == other.size;
562    }
563
564    @Override
565    public String toString() {
566      StringBuilder sb = new StringBuilder(32);
567      return sb.append("SnapshotWithSize:[").append(name).append(" ")
568        .append(StringUtils.byteDesc(size)).append("]").toString();
569    }
570  }
571
572  /**
573   * A reference to a collection of files in the archive directory for a single region.
574   */
575  static class StoreFileReference {
576    private final String regionName;
577    private final Multimap<String, String> familyToFiles;
578
579    StoreFileReference(String regionName) {
580      this.regionName = Objects.requireNonNull(regionName);
581      familyToFiles = HashMultimap.create();
582    }
583
584    String getRegionName() {
585      return regionName;
586    }
587
588    Multimap<String, String> getFamilyToFilesMapping() {
589      return familyToFiles;
590    }
591
592    void addFamilyStoreFile(String family, String storeFileName) {
593      familyToFiles.put(family, storeFileName);
594    }
595
596    @Override
597    public int hashCode() {
598      return new HashCodeBuilder().append(regionName).append(familyToFiles).toHashCode();
599    }
600
601    @Override
602    public boolean equals(Object o) {
603      if (this == o) {
604        return true;
605      }
606      if (!(o instanceof StoreFileReference)) {
607        return false;
608      }
609      StoreFileReference other = (StoreFileReference) o;
610      return regionName.equals(other.regionName) && familyToFiles.equals(other.familyToFiles);
611    }
612
613    @Override
614    public String toString() {
615      StringBuilder sb = new StringBuilder();
616      return sb.append("StoreFileReference[region=").append(regionName).append(", files=")
617        .append(familyToFiles).append("]").toString();
618    }
619  }
620}