-
Notifications
You must be signed in to change notification settings - Fork 396
/
Copy pathFileUtils.java
222 lines (201 loc) · 7.44 KB
/
FileUtils.java
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
/**
* Copyright 2015 Confluent Inc.
*
* Licensed 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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 io.confluent.connect.hdfs;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.kafka.common.TopicPartition;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import java.util.regex.Matcher;
import io.confluent.connect.hdfs.filter.CommittedFileFilter;
import io.confluent.connect.hdfs.storage.Storage;
public class FileUtils {
private static final Logger log = LoggerFactory.getLogger(FileUtils.class);
public static String logFileName(String url, String logsDir, TopicPartition topicPart) {
return fileName(url, logsDir, topicPart, "log");
}
public static String directoryName(String url, String topicsDir, TopicPartition topicPart) {
String topic = topicPart.topic();
int partition = topicPart.partition();
return url + "/" + topicsDir + "/" + topic + "/" + partition;
}
public static String directoryName(String url, String topicsDir, String directory) {
return url + "/" + topicsDir + "/" + directory;
}
public static String fileName(
String url,
String topicsDir,
TopicPartition topicPart,
String name
) {
String topic = topicPart.topic();
int partition = topicPart.partition();
return url + "/" + topicsDir + "/" + topic + "/" + partition + "/" + name;
}
public static String fileName(String url, String topicsDir, String directory, String name) {
return url + "/" + topicsDir + "/" + directory + "/" + name;
}
public static String tempFileName(
String url,
String topicsDir,
String directory,
String extension
) {
UUID id = UUID.randomUUID();
String name = id.toString() + "_" + "tmp" + extension;
return fileName(url, topicsDir, directory, name);
}
public static String committedFileName(
String url,
String topicsDir,
String directory,
TopicPartition topicPart,
long startOffset,
long endOffset,
String extension,
String zeroPadFormat
) {
String topic = topicPart.topic();
int partition = topicPart.partition();
StringBuilder sb = new StringBuilder();
sb.append(topic);
sb.append(HdfsSinkConnectorConstants.COMMMITTED_FILENAME_SEPARATOR);
sb.append(partition);
sb.append(HdfsSinkConnectorConstants.COMMMITTED_FILENAME_SEPARATOR);
sb.append(String.format(zeroPadFormat, startOffset));
sb.append(HdfsSinkConnectorConstants.COMMMITTED_FILENAME_SEPARATOR);
sb.append(String.format(zeroPadFormat, endOffset));
sb.append(extension);
String name = sb.toString();
return fileName(url, topicsDir, directory, name);
}
public static String topicDirectory(String url, String topicsDir, String topic) {
return url + "/" + topicsDir + "/" + topic;
}
public static FileStatus fileStatusWithMaxOffset(
Storage storage,
Path path,
CommittedFileFilter filter
) {
if (!storage.exists(path.toString())) {
return null;
}
long maxOffset = -1L;
FileStatus fileStatusWithMaxOffset = null;
List<FileStatus> statuses = storage.list(path.toString());
for (FileStatus status : statuses) {
if (status.isDirectory()) {
FileStatus fileStatus = fileStatusWithMaxOffset(storage, status.getPath(), filter);
if (fileStatus != null) {
long offset = extractOffset(fileStatus.getPath().getName());
if (offset > maxOffset) {
maxOffset = offset;
fileStatusWithMaxOffset = fileStatus;
}
}
} else {
String filename = status.getPath().getName();
log.trace("Checked for max offset: {}", status.getPath());
if (filter.accept(status.getPath())) {
long offset = extractOffset(filename);
if (offset > maxOffset) {
maxOffset = offset;
fileStatusWithMaxOffset = status;
}
}
}
}
return fileStatusWithMaxOffset;
}
public static long extractOffset(String filename) {
Matcher m = HdfsSinkConnectorConstants.COMMITTED_FILENAME_PATTERN.matcher(filename);
// NB: if statement has side effect of enabling group() call
if (!m.matches()) {
throw new IllegalArgumentException(filename + " does not match COMMITTED_FILENAME_PATTERN");
}
return Long.parseLong(m.group(HdfsSinkConnectorConstants.PATTERN_END_OFFSET_GROUP));
}
private static ArrayList<FileStatus> getDirectoriesImpl(Storage storage, Path path) {
List<FileStatus> statuses = storage.list(path.toString());
ArrayList<FileStatus> result = new ArrayList<>();
for (FileStatus status : statuses) {
if (status.isDirectory()) {
int count = 0;
List<FileStatus> fileStatuses = storage.list(status.getPath().toString());
for (FileStatus fileStatus : fileStatuses) {
if (fileStatus.isDirectory()) {
result.addAll(getDirectoriesImpl(storage, fileStatus.getPath()));
} else {
count++;
}
}
if (count == fileStatuses.size()) {
result.add(status);
}
}
}
return result;
}
public static FileStatus[] getDirectories(Storage storage, Path path) throws IOException {
ArrayList<FileStatus> result = getDirectoriesImpl(storage, path);
return result.toArray(new FileStatus[result.size()]);
}
private static ArrayList<FileStatus> traverseImpl(Storage storage, Path path, PathFilter filter) {
if (!storage.exists(path.toString())) {
return new ArrayList<>();
}
ArrayList<FileStatus> result = new ArrayList<>();
List<FileStatus> statuses = storage.list(path.toString());
for (FileStatus status : statuses) {
if (status.isDirectory()) {
result.addAll(traverseImpl(storage, status.getPath(), filter));
} else {
if (filter.accept(status.getPath())) {
result.add(status);
}
}
}
return result;
}
private static ArrayList<FileStatus> traverseImpl(FileSystem fs, Path path) throws IOException {
if (!fs.exists(path)) {
return new ArrayList<>();
}
ArrayList<FileStatus> result = new ArrayList<>();
FileStatus[] statuses = fs.listStatus(path);
for (FileStatus status : statuses) {
if (status.isDirectory()) {
result.addAll(traverseImpl(fs, status.getPath()));
} else {
result.add(status);
}
}
return result;
}
public static FileStatus[] traverse(Storage storage, Path path, PathFilter filter)
throws IOException {
ArrayList<FileStatus> result = traverseImpl(storage, path, filter);
return result.toArray(new FileStatus[result.size()]);
}
public static FileStatus[] traverse(FileSystem fs, Path path) throws IOException {
ArrayList<FileStatus> result = traverseImpl(fs, path);
return result.toArray(new FileStatus[result.size()]);
}
}