-
Notifications
You must be signed in to change notification settings - Fork 1
Expand file tree
/
Copy pathtask2.patch
More file actions
455 lines (446 loc) · 19.1 KB
/
task2.patch
File metadata and controls
455 lines (446 loc) · 19.1 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
diff --git a/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/UtilizationSchemaDefinition.java b/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/UtilizationSchemaDefinition.java
index 193ee758f..d33c7daf3 100644
--- a/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/UtilizationSchemaDefinition.java
+++ b/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/UtilizationSchemaDefinition.java
@@ -51,6 +51,8 @@ public class UtilizationSchemaDefinition implements ReconSchemaDefinition {
"CLUSTER_GROWTH_DAILY";
public static final String FILE_COUNT_BY_SIZE_TABLE_NAME =
"FILE_COUNT_BY_SIZE";
+ public static final String CONTAINER_COUNT_BY_SIZE_TABLE_NAME =
+ "CONTAINER_COUNT_BY_SIZE";
@Inject
UtilizationSchemaDefinition(DataSource dataSource) {
@@ -68,6 +70,9 @@ public void initializeSchema() throws SQLException {
if (!TABLE_EXISTS_CHECK.test(conn, CLUSTER_GROWTH_DAILY_TABLE_NAME)) {
createClusterGrowthTable();
}
+ if (!TABLE_EXISTS_CHECK.test(conn, CONTAINER_COUNT_BY_SIZE_TABLE_NAME)) {
+ createContainerSizeCountTable();
+ }
}
private void createClusterGrowthTable() {
@@ -96,6 +101,15 @@ private void createFileSizeCountTable() {
.execute();
}
+ private void createContainerSizeCountTable() {
+ dslContext.createTableIfNotExists(CONTAINER_COUNT_BY_SIZE_TABLE_NAME)
+ .column("container_size", SQLDataType.BIGINT.nullable(false))
+ .column("count", SQLDataType.BIGINT)
+ .constraint(DSL.constraint("pk_container_size")
+ .primaryKey("container_size"))
+ .execute();
+ }
+
/**
* Returns the DSL context.
*
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
index 51499a0d6..7ac27f382 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
@@ -86,6 +86,7 @@
import org.apache.hadoop.ozone.recon.persistence.ContainerHealthSchemaManager;
import org.apache.hadoop.ozone.recon.spi.ReconContainerMetadataManager;
import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.apache.hadoop.ozone.recon.tasks.ContainerSizeCountTask;
import org.apache.hadoop.ozone.recon.tasks.ReconTaskConfig;
import com.google.inject.Inject;
import static org.apache.hadoop.hdds.recon.ReconConfigKeys.RECON_SCM_CONFIG_PREFIX;
@@ -98,7 +99,9 @@
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.IncrementalContainerReportFromDatanode;
import org.apache.ratis.util.ExitUtils;
+import org.hadoop.ozone.recon.schema.tables.daos.ContainerCountBySizeDao;
import org.hadoop.ozone.recon.schema.tables.daos.ReconTaskStatusDao;
+import org.jooq.DSLContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -137,6 +140,8 @@
public ReconStorageContainerManagerFacade(OzoneConfiguration conf,
StorageContainerServiceProvider scmServiceProvider,
ReconTaskStatusDao reconTaskStatusDao,
+ ContainerCountBySizeDao containerCountBySizeDao,
+ DSLContext dslContext,
ContainerHealthSchemaManager containerHealthSchemaManager,
ReconContainerMetadataManager reconContainerMetadataManager,
ReconUtils reconUtils) throws IOException {
@@ -281,6 +286,13 @@ public ReconStorageContainerManagerFacade(OzoneConfiguration conf,
reconTaskStatusDao, containerHealthSchemaManager,
containerPlacementPolicy,
reconTaskConfig));
+ reconScmTasks.add(new ContainerSizeCountTask(
+ containerManager,
+ scmServiceProvider,
+ reconTaskStatusDao,
+ reconTaskConfig,
+ containerCountBySizeDao,
+ dslContext));
}
/**
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java
index 994c0287a..931519323 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java
@@ -1,108 +1,173 @@
-///*
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements. See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership. The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License. You may obtain a copy of the License at
-// * <p>
-// * http://www.apache.org/licenses/LICENSE-2.0
-// * <p>
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//package org.apache.hadoop.ozone.recon.tasks;
-//
-//
-//import org.apache.hadoop.hdds.scm.PlacementPolicy;
-//import org.apache.hadoop.hdds.scm.container.ContainerInfo;
-//import org.apache.hadoop.hdds.scm.container.ContainerManager;
-//import org.apache.hadoop.ozone.recon.fsck.ContainerHealthTask;
-//import org.apache.hadoop.ozone.recon.persistence.ContainerHealthSchemaManager;
-//import org.apache.hadoop.ozone.recon.scm.ReconScmTask;
-//
-//import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
-//import org.apache.hadoop.util.Time;
-//import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
-//import org.hadoop.ozone.recon.schema.tables.daos.ContainerCountBySizeDao;
-//import org.hadoop.ozone.recon.schema.tables.daos.ReconTaskStatusDao;
-//import org.slf4j.Logger;
-//import org.slf4j.LoggerFactory;
-//
-//import java.util.*;
-//
-//import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
-//
-//
-///**
-// * Class that scans the list of containers and keeps track of container sizes
-// * binned into ranges (1KB, 2Kb..,4MB,.., 1TB,..1PB) to the Recon
-// * containerSize DB.
-// */
-//public class ContainerSizeCountTask extends ReconScmTask {
-//
-// private static final Logger LOG =
-// LoggerFactory.getLogger(ContainerSizeCountTask.class);
-//
-// private StorageContainerServiceProvider scmClient;
-// private ContainerManager containerManager;
-// private final long interval;
-//
-// private Set<ContainerInfo> processedContainers = new HashSet<>();
-//
-//
-// public ContainerSizeCountTask(
-// ContainerManager containerManager,
-// StorageContainerServiceProvider scmClient,
-// ReconTaskStatusDao reconTaskStatusDao,
-// ReconTaskConfig reconTaskConfig) {
-// super(reconTaskStatusDao);
-// this.scmClient = scmClient;
-// this.containerManager = containerManager;
-// interval = reconTaskConfig.getMissingContainerTaskInterval().toMillis();
-// }
-//
-//
-//// @Override
-//// protected void run() {
-//// try {
-//// while (canRun()) {
-//// wait(interval);
-//// long start = Time.monotonicNow();
-//// long currentTime = System.currentTimeMillis();
-//// long existingCount = processExistingDBRecords(currentTime);
-//// LOG.info("Container Size Count task thread took {} milliseconds to" +
-//// " process {} existing database records.",
-//// Time.monotonicNow() - start, existingCount);
-//// start = Time.monotonicNow();
-//// final List<ContainerInfo> containers = containerManager.getContainers();
-//// containers.stream()
-//// .filter(c -> !processedContainers.contains(c))
-//// .forEach(c -> processContainer(c, currentTime));
-//// recordSingleRunCompletion();
-//// LOG.info("Container Health task thread took {} milliseconds for" +
-//// " processing {} containers.", Time.monotonicNow() - start,
-//// containers.size());
-//// processedContainers.clear();
-//// }
-//// } catch (Throwable t) {
-//// LOG.error("Exception in Missing Container task Thread.", t);
-//// if (t instanceof InterruptedException) {
-//// Thread.currentThread().interrupt();
-//// }
-//// }
-//// }
-//
-// @Override
-// public String getTaskName() {
-// return "ContainerSizeCountTask";
-// }
-//
-//
-//
-//}
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.tasks;
+
+
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.scm.ReconScmTask;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.hadoop.ozone.recon.schema.tables.daos.ContainerCountBySizeDao;
+import org.hadoop.ozone.recon.schema.tables.daos.ReconTaskStatusDao;
+import org.hadoop.ozone.recon.schema.tables.pojos.ContainerCountBySize;
+import org.jooq.DSLContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+import static org.hadoop.ozone.recon.schema.tables.ContainerCountBySizeTable.CONTAINER_COUNT_BY_SIZE;
+
+
+/**
+ * Class that scans the list of containers and keeps track of container sizes
+ * binned into ranges (1KB, 2Kb..,4MB,.., 1TB,..1PB) to the Recon
+ * containerSize DB.
+ */
+public class ContainerSizeCountTask extends ReconScmTask {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ContainerSizeCountTask.class);
+
+ private StorageContainerServiceProvider scmClient;
+ private ContainerManager containerManager;
+ private final long interval;
+ private ContainerCountBySizeDao containerCountBySizeDao;
+ private DSLContext dslContext;
+
+
+ public ContainerSizeCountTask(
+ ContainerManager containerManager,
+ StorageContainerServiceProvider scmClient,
+ ReconTaskStatusDao reconTaskStatusDao,
+ ReconTaskConfig reconTaskConfig,
+ ContainerCountBySizeDao containerCountBySizeDao,
+ DSLContext dslContext) {
+ super(reconTaskStatusDao);
+ this.scmClient = scmClient;
+ this.containerManager = containerManager;
+ this.containerCountBySizeDao = containerCountBySizeDao;
+ this.dslContext = dslContext;
+ interval = reconTaskConfig.getContainerSizeCountTaskInterval().toMillis();
+ }
+
+
+ @Override
+ protected synchronized void run() {
+ try {
+ while (canRun()) {
+ wait(interval);
+
+ final List<ContainerInfo> containers = containerManager.getContainers();
+ Map<ContainerSizeCountKey, Long> containerSizeCountMap =
+ new HashMap<>();
+ for (int i = 0; i < containers.size(); i++) {
+ handlePutKeyEvent(containers.get(i), containerSizeCountMap);
+ }
+ // Truncate table before inserting new rows
+ int execute = dslContext.delete(CONTAINER_COUNT_BY_SIZE).execute();
+ LOG.info("Deleted {} records from {}", execute, CONTAINER_COUNT_BY_SIZE);
+
+ writeCountsToDB(true, containerSizeCountMap);
+
+ LOG.info("Completed a 'reprocess' run of ContainerSizeCountTask.");
+ }
+ } catch (Throwable t) {
+ LOG.error("Exception in Container Size Distribution task Thread.", t);
+ if (t instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+
+ /**
+ * Populate DB with the counts of container sizes calculated
+ * using the dao.
+ */
+ private void writeCountsToDB(boolean isDbTruncated,
+ Map<ContainerSizeCountKey, Long>
+ containerSizeCountMap) {
+ List<ContainerCountBySize> insertToDb = new ArrayList<>();
+
+ containerSizeCountMap.keySet().forEach((ContainerSizeCountKey key) -> {
+ ContainerCountBySize newRecord = new ContainerCountBySize();
+ newRecord.setContainerSize(key.containerSizeUpperBound);
+ newRecord.setCount(containerSizeCountMap.get(key));
+ if (!isDbTruncated) {
+ insertToDb.add(newRecord);
+ }
+ });
+ containerCountBySizeDao.insert(insertToDb);
+ }
+
+ @Override
+ public String getTaskName() {
+ return "ContainerSizeCountTask";
+ }
+
+ /**
+ * Calculate and update the count of containers being tracked by
+ * containerSizeCountMap.
+ * Used by reprocess() and process().
+ *
+ * @param containerInfo OmKey being updated for count
+ */
+ private void handlePutKeyEvent(ContainerInfo containerInfo,
+ Map<ContainerSizeCountKey, Long> containerSizeCountMap) {
+ ContainerSizeCountKey key = getContainerSizeCountKey(containerInfo);
+ Long count = containerSizeCountMap.containsKey(key) ?
+ containerSizeCountMap.get(key) + 1L : 1L;
+ containerSizeCountMap.put(key, count);
+ }
+
+ private ContainerSizeCountKey getContainerSizeCountKey(
+ ContainerInfo containerInfo) {
+ return new ContainerSizeCountKey(
+ ReconUtils.getFileSizeUpperBound(containerInfo.getUsedBytes()));
+ }
+
+
+ private static class ContainerSizeCountKey {
+
+ private Long containerSizeUpperBound;
+
+ ContainerSizeCountKey(
+ Long containerSizeUpperBound) {
+ this.containerSizeUpperBound = containerSizeUpperBound;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof ContainerSizeCountKey) {
+ ContainerSizeCountTask.ContainerSizeCountKey
+ s = (ContainerSizeCountTask.ContainerSizeCountKey) obj;
+ return
+ containerSizeUpperBound.equals(s.containerSizeUpperBound);
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return (containerSizeUpperBound).hashCode();
+ }
+ }
+
+}
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java
index 03ff12349..8c1a244aa 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java
@@ -80,9 +80,13 @@ public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
omMetadataManager.getKeyTable(getBucketLayout());
Map<FileSizeCountKey, Long> fileSizeCountMap = new HashMap<>();
try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+ // get the iterator for omKeyInfoTable
keyIter = omKeyInfoTable.iterator()) {
+
+ // Iterate the table one by one and fill up the map
while (keyIter.hasNext()) {
Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
+ // It puts the upper bound value along with the fileSizeCountKey value
handlePutKeyEvent(kv.getValue(), fileSizeCountMap);
}
} catch (IOException ioEx) {
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskConfig.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskConfig.java
index 9788bf67e..709d82ede 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskConfig.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskConfig.java
@@ -66,4 +66,21 @@ public void setMissingContainerTaskInterval(Duration interval) {
this.missingContainerTaskInterval = interval.toMillis();
}
+ @Config(key = "containercounttask.interval",
+ type = ConfigType.TIME,
+ defaultValue = "300s",
+ tags = { ConfigTag.RECON, ConfigTag.OZONE },
+ description = "The time interval of the periodic check for " +
+ "container size distribution in the cluster as reported "
+ )
+ private long containerSizeCountTaskInterval = Duration.ofMinutes(5).toMillis();
+
+ public Duration getContainerSizeCountTaskInterval() {
+ return Duration.ofMillis(missingContainerTaskInterval);
+ }
+
+ public void setContainerSizeCountTaskInterval(Duration interval) {
+ this.missingContainerTaskInterval = interval.toMillis();
+ }
+
}
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/TestUtilizationSchemaDefinition.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/TestUtilizationSchemaDefinition.java
index 3c9c19958..09e590410 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/TestUtilizationSchemaDefinition.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/persistence/TestUtilizationSchemaDefinition.java
@@ -17,8 +17,7 @@
*/
package org.apache.hadoop.ozone.recon.persistence;
-import static org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition.CLUSTER_GROWTH_DAILY_TABLE_NAME;
-import static org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition.FILE_COUNT_BY_SIZE_TABLE_NAME;
+import static org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition.*;
import static org.hadoop.ozone.recon.schema.tables.ClusterGrowthDailyTable.CLUSTER_GROWTH_DAILY;
import static org.hadoop.ozone.recon.schema.tables.FileCountBySizeTable.FILE_COUNT_BY_SIZE;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -103,6 +102,26 @@ public void testReconSchemaCreated() throws Exception {
"Unexpected number of columns");
assertEquals(expectedPairsFileCount, actualPairsFileCount,
"Columns Do not Match ");
+
+
+ ResultSet resultSetContainerCount = metaData.getColumns(null, null,
+ CONTAINER_COUNT_BY_SIZE_TABLE_NAME, null);
+
+ List<Pair<String, Integer>> expectedPairsContainerCount = new ArrayList<>();
+ expectedPairsContainerCount.add(
+ new ImmutablePair<>("container_size", Types.BIGINT));
+ expectedPairsContainerCount.add(
+ new ImmutablePair<>("count", Types.BIGINT));
+
+ List<Pair<String, Integer>> actualPairsContainerCount = new ArrayList<>();
+ while (resultSetContainerCount.next()) {
+ actualPairsContainerCount.add(new ImmutablePair<>(resultSetContainerCount.getString(
+ "COLUMN_NAME"), resultSetContainerCount.getInt(
+ "DATA_TYPE")));
+ }
+ assertEquals(2, actualPairsContainerCount.size(),
+ "Unexpected number of columns");
+
}
@Test