1 /*
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
9 *
10 * http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
17 */
18
19 package org.apache.hadoop.hbase.master;
20
21 import com.google.common.base.Preconditions;
22 import java.io.IOException;
23 import java.util.concurrent.atomic.AtomicBoolean;
24 import java.util.concurrent.atomic.AtomicInteger;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.fs.FileSystem;
27 import org.apache.hadoop.fs.Path;
28 import org.apache.hadoop.hbase.ClusterId;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.hbase.util.FSUtils;
31 import org.slf4j.Logger;
32 import org.slf4j.LoggerFactory;
33
34 /**
35 * Caches the cluster ID of the cluster. For standby masters, this is used to serve the client
36 * RPCs that fetch the cluster ID. ClusterID is only created by an active master if one does not
37 * already exist. Standby masters just read the information from the file system. This class is
38 * thread-safe.
39 *
40 * TODO: Make it a singleton without affecting concurrent junit tests.
41 */
42 @InterfaceAudience.Private
43 public class CachedClusterId {
44
45 public static final Logger LOG = LoggerFactory.getLogger(CachedClusterId.class);
46 private static final int MAX_FETCH_TIMEOUT_MS = 10000;
47
48 private Path rootDir;
49 private FileSystem fs;
50
51 // When true, indicates that a FileSystem fetch of ClusterID is in progress. This is used to
52 // avoid multiple fetches from FS and let only one thread fetch the information.
53 AtomicBoolean fetchInProgress = new AtomicBoolean(false);
54
55 // When true, it means that the cluster ID has been fetched successfully from fs.
56 private AtomicBoolean isClusterIdSet = new AtomicBoolean(false);
57 // Immutable once set and read multiple times.
58 private ClusterId clusterId;
59
60 // cache stats for testing.
61 private AtomicInteger cacheMisses = new AtomicInteger(0);
62
63 public CachedClusterId(Configuration conf) throws IOException {
64 rootDir = FSUtils.getRootDir(conf);
65 fs = rootDir.getFileSystem(conf);
66 }
67
68 /**
69 * Succeeds only once, when setting to a non-null value. Overwrites are not allowed.
70 */
71 private void setClusterId(ClusterId id) {
72 if (id == null || isClusterIdSet.get()) {
73 return;
74 }
75 clusterId = id;
76 isClusterIdSet.set(true);
77 }
78
79 /**
80 * Returns a cached copy of the cluster ID. null if the cache is not populated.
81 */
82 private String getClusterId() {
83 if (!isClusterIdSet.get()) {
84 return null;
85 }
86 // It is ok to read without a lock since clusterId is immutable once set.
87 return clusterId.toString();
88 }
89
90 /**
91 * Attempts to fetch the cluster ID from the file system. If no attempt is already in progress,
92 * synchronously fetches the cluster ID and sets it. If an attempt is already in progress,
93 * returns right away and the caller is expected to wait for the fetch to finish.
94 * @return true if the attempt is done, false if another thread is already fetching it.
95 */
96 private boolean attemptFetch() {
97 if (fetchInProgress.compareAndSet(false, true)) {
98 // A fetch is not in progress, so try fetching the cluster ID synchronously and then notify
99 // the waiting threads.
100 try {
101 cacheMisses.incrementAndGet();
102 setClusterId(FSUtils.getClusterId(fs, rootDir));
103 } catch (IOException e) {
104 LOG.warn("Error fetching cluster ID", e);
105 } finally {
106 Preconditions.checkState(fetchInProgress.compareAndSet(true, false));
107 synchronized (fetchInProgress) {
108 fetchInProgress.notifyAll();
109 }
110 }
111 return true;
112 }
113 return false;
114 }
115
116 private void waitForFetchToFinish() throws InterruptedException {
117 synchronized (fetchInProgress) {
118 while (fetchInProgress.get()) {
119 // We don't want the fetches to block forever, for example if there are bugs
120 // of missing notifications.
121 fetchInProgress.wait(MAX_FETCH_TIMEOUT_MS);
122 }
123 }
124 }
125
126 /**
127 * Fetches the ClusterId from FS if it is not cached locally. Atomically updates the cached
128 * copy and is thread-safe. Optimized to do a single fetch when there are multiple threads are
129 * trying get from a clean cache.
130 *
131 * @return ClusterId by reading from FileSystem or null in any error case or cluster ID does
132 * not exist on the file system.
133 */
134 public String getFromCacheOrFetch() {
135 String id = getClusterId();
136 if (id != null) {
137 return id;
138 }
139 if (!attemptFetch()) {
140 // A fetch is in progress.
141 try {
142 waitForFetchToFinish();
143 } catch (InterruptedException e) {
144 // pass and return whatever is in the cache.
145 }
146 }
147 return getClusterId();
148 }
149
150 public int getCacheStats() {
151 return cacheMisses.get();
152 }
153 }