1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.util;
21
22 import java.io.IOException;
23 import java.io.InterruptedIOException;
24 import java.util.ArrayList;
25 import java.util.Collections;
26 import java.util.List;
27 import java.util.concurrent.Callable;
28 import java.util.concurrent.CompletionService;
29 import java.util.concurrent.ExecutionException;
30 import java.util.concurrent.ExecutorCompletionService;
31 import java.util.concurrent.Future;
32 import java.util.concurrent.ThreadFactory;
33 import java.util.concurrent.ThreadPoolExecutor;
34 import java.util.concurrent.TimeUnit;
35
36 import org.apache.commons.logging.Log;
37 import org.apache.commons.logging.LogFactory;
38
39 import org.apache.hadoop.classification.InterfaceAudience;
40 import org.apache.hadoop.conf.Configuration;
41 import org.apache.hadoop.fs.FileSystem;
42 import org.apache.hadoop.fs.Path;
43 import org.apache.hadoop.hbase.HRegionInfo;
44 import org.apache.hadoop.hbase.HTableDescriptor;
45 import org.apache.hadoop.hbase.backup.HFileArchiver;
46 import org.apache.hadoop.hbase.catalog.CatalogTracker;
47 import org.apache.hadoop.hbase.catalog.MetaEditor;
48 import org.apache.hadoop.hbase.client.Delete;
49 import org.apache.hadoop.hbase.master.AssignmentManager;
50 import org.apache.hadoop.hbase.regionserver.HRegion;
51
52
53
54
55 @InterfaceAudience.Private
56 public abstract class ModifyRegionUtils {
57 private static final Log LOG = LogFactory.getLog(ModifyRegionUtils.class);
58
59 private ModifyRegionUtils() {
60 }
61
62 public interface RegionFillTask {
63 public void fillRegion(final HRegion region) throws IOException;
64 }
65
66
67
68
69
70
71
72
73
74
75
76 public static List<HRegionInfo> createRegions(final Configuration conf, final Path rootDir,
77 final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions) throws IOException {
78 return createRegions(conf, rootDir, hTableDescriptor, newRegions, null);
79 }
80
81
82
83
84
85
86
87
88
89
90
91
92 public static List<HRegionInfo> createRegions(final Configuration conf, final Path rootDir,
93 final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
94 final RegionFillTask task) throws IOException {
95 if (newRegions == null) return null;
96 int regionNumber = newRegions.length;
97 ThreadPoolExecutor regionOpenAndInitThreadPool = getRegionOpenAndInitThreadPool(conf,
98 "RegionOpenAndInitThread-" + hTableDescriptor.getNameAsString(), regionNumber);
99 CompletionService<HRegionInfo> completionService = new ExecutorCompletionService<HRegionInfo>(
100 regionOpenAndInitThreadPool);
101 List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
102 for (final HRegionInfo newRegion : newRegions) {
103 completionService.submit(new Callable<HRegionInfo>() {
104 public HRegionInfo call() throws IOException {
105
106 HRegion region = HRegion.createHRegion(newRegion,
107 rootDir, conf, hTableDescriptor, null,
108 false, true);
109 try {
110
111 if (task != null) {
112 task.fillRegion(region);
113 }
114 } finally {
115
116 region.close();
117 }
118 return region.getRegionInfo();
119 }
120 });
121 }
122 try {
123
124 for (int i = 0; i < regionNumber; i++) {
125 Future<HRegionInfo> future = completionService.take();
126 HRegionInfo regionInfo = future.get();
127 regionInfos.add(regionInfo);
128 }
129 } catch (InterruptedException e) {
130 LOG.error("Caught " + e + " during region creation");
131 throw new InterruptedIOException(e.getMessage());
132 } catch (ExecutionException e) {
133 throw new IOException(e);
134 } finally {
135 regionOpenAndInitThreadPool.shutdownNow();
136 }
137 return regionInfos;
138 }
139
140
141
142
143
144 static ThreadPoolExecutor getRegionOpenAndInitThreadPool(final Configuration conf,
145 final String threadNamePrefix, int regionNumber) {
146 int maxThreads = Math.min(regionNumber, conf.getInt(
147 "hbase.hregion.open.and.init.threads.max", 10));
148 ThreadPoolExecutor regionOpenAndInitThreadPool = Threads
149 .getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
150 new ThreadFactory() {
151 private int count = 1;
152
153 public Thread newThread(Runnable r) {
154 Thread t = new Thread(r, threadNamePrefix + "-" + count++);
155 return t;
156 }
157 });
158 return regionOpenAndInitThreadPool;
159 }
160
161
162
163
164
165
166
167 public static void assignRegions(final AssignmentManager assignmentManager,
168 final List<HRegionInfo> regions) throws IOException {
169 try {
170 assignmentManager.getRegionStates().createRegionStates(regions);
171 assignmentManager.assign(regions);
172 } catch (InterruptedException e) {
173 LOG.error("Caught " + e + " during round-robin assignment");
174 throw new InterruptedIOException(e.getMessage());
175 }
176 }
177 }