Skip to content

Commit a564aa4

Browse files
committed
HADOOP-19161. FlagSet class.
- thread safe - can be made immutable. - toString prints {values, like, this} - tests Feels ready to review now; nothing more I can think of Change-Id: Ia7251de59b31bf2e682f79fe9c6ddc095ec4bf92
1 parent f3571ac commit a564aa4

File tree

3 files changed

+172
-43
lines changed

3 files changed

+172
-43
lines changed

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FlagSet.java

Lines changed: 45 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -18,18 +18,27 @@
1818

1919
package org.apache.hadoop.fs.impl;
2020

21+
import java.util.Collections;
2122
import java.util.Map;
2223
import java.util.Set;
24+
import java.util.concurrent.atomic.AtomicBoolean;
25+
import java.util.stream.Collectors;
2326

2427
import org.apache.hadoop.conf.Configuration;
2528
import org.apache.hadoop.fs.StreamCapabilities;
29+
import org.apache.hadoop.util.Preconditions;
2630

2731
import static java.util.Objects.requireNonNull;
2832
import static org.apache.hadoop.util.ConfigurationUtil.mapEnumNamesToValues;
2933

3034
/**
31-
* A set of flags, constructed from a configuration option/value
35+
* A set of flags, constructed from a configuration option or from a string,
36+
* with the semantics of
37+
* {@link org.apache.hadoop.util.ConfigurationUtil#parseEnumSet(String, String, Class, boolean)}
3238
* and implementing {@link StreamCapabilities}.
39+
* Once declared immutable, flags cannot be changed.
40+
* While mutable, all getters and setters are synchronized, so
41+
* an instance is always thread-safe.
3342
*/
3443
public final class FlagSet<E extends Enum<E>> implements StreamCapabilities {
3544

@@ -38,6 +47,11 @@ public final class FlagSet<E extends Enum<E>> implements StreamCapabilities {
3847
*/
3948
private final Set<E> flags;
4049

50+
/**
51+
* Is the set immutable?
52+
*/
53+
private final AtomicBoolean immutable = new AtomicBoolean(false);
54+
4155
/**
4256
* Mapping of prefixed flag names to enum values.
4357
*/
@@ -55,36 +69,46 @@ private FlagSet(final Class<E> enumClass, final String prefix, final Set<E> flag
5569
}
5670

5771
/**
58-
* Get the flags.
59-
* This is mutable.
72+
* Get a copy of the flags.
73+
* This is immutable.
6074
* @return the flags.
6175
*/
62-
public Set<E> flags() {
63-
return flags;
76+
public synchronized Set<E> flags() {
77+
return Collections.unmodifiableSet(flags);
6478
}
6579

6680
/**
6781
* Is a flag enabled?
6882
* @param flag flag to check
6983
* @return true if it is in the set of enabled flags.
7084
*/
71-
public boolean enabled(final E flag) {
85+
public synchronized boolean enabled(final E flag) {
7286
return flags.contains(flag);
7387
}
7488

89+
/**
90+
* Check for mutability before any mutating operation.
91+
*/
92+
private void checkMutable() {
93+
Preconditions.checkState(!immutable.get(),
94+
"FlagSet is immutable");
95+
}
96+
7597
/**
7698
* Enable a flag.
7799
* @param flag flag to enable.
78100
*/
79-
public void enable(final E flag) {
101+
public synchronized void enable(final E flag) {
102+
checkMutable();
80103
flags.add(flag);
81104
}
82105

83106
/**
84107
* Disable a flag.
85108
* @param flag flag to disable
86109
*/
87-
public void disable(final E flag) {
110+
public synchronized void disable(final E flag) {
111+
checkMutable();
88112
flags.remove(flag);
89113
}
90114

@@ -102,7 +126,7 @@ public void set(final E flag, boolean state) {
102126
}
103127

104128
/**
105-
* Does is a flag enabled.
129+
* Is a flag enabled?
106130
* @param capability string to query the stream support for.
107131
* @return true if the capability maps to an enum value and
108132
* that value is set.
@@ -113,11 +137,20 @@ public boolean hasCapability(final String capability) {
113137
return e != null && enabled(e);
114138
}
115139

140+
/**
141+
* Make immutable; no-op if already set.
142+
*/
143+
public synchronized void makeImmutable() {
144+
immutable.set(true);
145+
}
146+
116147
@Override
117148
public String toString() {
118-
return "FlagSet{" +
119-
flags +
120-
'}';
149+
return "{" +
150+
(flags.stream()
151+
.map(e -> e.name())
152+
.collect(Collectors.joining(", ")))
153+
+ '}';
121154
}
122155

123156
/**

hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestFlagSet.java

Lines changed: 125 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -16,23 +16,7 @@
1616
* limitations under the License.
1717
*/
1818

19-
package org.apache.hadoop.fs.impl;/*
20-
* Licensed to the Apache Software Foundation (ASF) under one
21-
* or more contributor license agreements. See the NOTICE file
22-
* distributed with this work for additional information
23-
* regarding copyright ownership. The ASF licenses this file
24-
* to you under the Apache License, Version 2.0 (the
25-
* "License"); you may not use this file except in compliance
26-
* with the License. You may obtain a copy of the License at
27-
*
28-
* http://www.apache.org/licenses/LICENSE-2.0
29-
*
30-
* Unless required by applicable law or agreed to in writing, software
31-
* distributed under the License is distributed on an "AS IS" BASIS,
32-
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
33-
* See the License for the specific language governing permissions and
34-
* limitations under the License.
35-
*/
19+
package org.apache.hadoop.fs.impl;
3620

3721
import org.assertj.core.api.Assertions;
3822
import org.junit.Test;
@@ -52,17 +36,23 @@ public class TestFlagSet extends AbstractHadoopTestBase {
5236

5337
public static final String KEY = "key";
5438

39+
/**
40+
* Flagset used in tests and assertions.
41+
*/
5542
private FlagSet<SimpleEnum> flagSet =
5643
createFlagSet(SimpleEnum.class, KEY + ".", noneOf(SimpleEnum.class));
5744

5845
/**
5946
* Simple Enums.
6047
* "i" is included for case tests, as it is special in turkey.
6148
*/
62-
private enum SimpleEnum { a, b }
49+
private enum SimpleEnum { a, b, c }
6350

51+
/**
52+
* Test that an entry can be enabled and disabled.
53+
*/
6454
@Test
65-
public void testSingleEntry() throws Throwable {
55+
public void testEntryEnableDisable() throws Throwable {
6656
Assertions.assertThat(flagSet.flags()).isEmpty();
6757
assertDisabled(SimpleEnum.a);
6858
flagSet.enable(SimpleEnum.a);
@@ -71,73 +61,177 @@ public void testSingleEntry() throws Throwable {
7161
assertDisabled(SimpleEnum.a);
7262
}
7363

64+
/**
65+
* Test the setter.
66+
*/
7467
@Test
7568
public void testSetMethod() throws Throwable {
7669
Assertions.assertThat(flagSet.flags()).isEmpty();
7770
flagSet.set(SimpleEnum.a, true);
7871
assertEnabled(SimpleEnum.a);
7972
flagSet.set(SimpleEnum.a, false);
80-
flagSet.disable(SimpleEnum.a);
73+
assertDisabled(SimpleEnum.a);
74+
}
75+
76+
/**
77+
* Test mutability by making immutable and
78+
* expecting setters to fail.
79+
*/
80+
@Test
81+
public void testMutability() throws Throwable {
82+
flagSet.set(SimpleEnum.a, true);
83+
flagSet.makeImmutable();
84+
intercept(IllegalStateException.class, () ->
85+
flagSet.disable(SimpleEnum.a));
86+
assertEnabled(SimpleEnum.a);
87+
intercept(IllegalStateException.class, () ->
88+
flagSet.set(SimpleEnum.a, false));
89+
assertEnabled(SimpleEnum.a);
90+
// now look at the setters
91+
intercept(IllegalStateException.class, () ->
92+
flagSet.enable(SimpleEnum.b));
93+
assertDisabled(SimpleEnum.b);
94+
intercept(IllegalStateException.class, () ->
95+
flagSet.set(SimpleEnum.b, true));
96+
assertDisabled(SimpleEnum.b);
97+
}
98+
99+
/**
100+
* Test stringification.
101+
*/
102+
@Test
103+
public void testToString() throws Throwable {
104+
assertStringValue("{}");
105+
flagSet.enable(SimpleEnum.a);
106+
assertStringValue("{a}");
107+
flagSet.enable(SimpleEnum.b);
108+
assertStringValue("{a, b}");
109+
}
110+
111+
/**
112+
* Assert that {@link FlagSet#toString()} matches the expected
113+
* value.
114+
* @param expected expected value
115+
*/
116+
private void assertStringValue(final String expected) {
117+
Assertions.assertThat(flagSet.toString())
118+
.isEqualTo(expected);
81119
}
82120

121+
/**
122+
* Test parsing from a configuration file.
123+
* Multiple entries must be parsed, whitespace trimmed.
124+
*/
83125
@Test
84126
public void testConfEntry() throws Throwable {
127+
final Configuration conf = mkConf("a\t,\nc ");
128+
flagSet = buildFlagSet(SimpleEnum.class, conf, KEY, true);
129+
assertEnabled(SimpleEnum.a);
130+
assertEnabled(SimpleEnum.c);
131+
assertDisabled(SimpleEnum.b);
132+
assertHasCapability(KEY + ".a");
133+
assertHasCapability(KEY + ".c");
134+
assertLacksCapability(KEY + ".b");
135+
}
136+
137+
/**
138+
* Test parsing from a configuration file,
139+
* where an entry is unknown: getter set to ignore
140+
*/
141+
@Test
142+
public void testConfEntryWithUnknownIgnored() throws Throwable {
85143
final Configuration conf = mkConf("a, unknown");
86144
flagSet = buildFlagSet(SimpleEnum.class, conf, KEY, true);
87145
assertEnabled(SimpleEnum.a);
88146
assertDisabled(SimpleEnum.b);
89-
assertCapabilityTrue(KEY + ".a");
90-
assertCapabilityFalse(KEY + ".b");
147+
assertHasCapability(KEY + ".a");
148+
assertLacksCapability(KEY + ".b");
91149
}
92150

151+
/**
152+
* Test parsing from a configuration file where
153+
* the same entry is duplicated.
154+
*/
93155
@Test
94-
public void testConfUnknown() throws Throwable {
156+
public void testDuplicateConfEntry() throws Throwable {
157+
final Configuration conf = mkConf("a,\ta,\na");
158+
flagSet = buildFlagSet(SimpleEnum.class, conf, KEY, true);
159+
assertEnabled(SimpleEnum.a);
160+
assertHasCapability(KEY + ".a");
161+
}
162+
163+
/**
164+
* Handle an unknown configuration value.
165+
*/
166+
@Test
167+
public void testConfUnknownFailure() throws Throwable {
95168
intercept(IllegalArgumentException.class, () ->
96169
buildFlagSet(SimpleEnum.class, mkConf("a, unknown"), KEY, false));
97170
}
98171

172+
/**
173+
* Create a configuration with {@link #KEY} set to the given value.
174+
* @param value value to set
175+
* @return the configuration.
176+
*/
99177
private static Configuration mkConf(final String value) {
100178
final Configuration conf = new Configuration(false);
101179
conf.set(KEY, value);
102180
return conf;
103181
}
104182

105-
private void assertCapabilityTrue(final String capability) {
183+
/**
184+
* Assert that the flagset has a capability.
185+
* @param capability capability to probe for
186+
*/
187+
private void assertHasCapability(final String capability) {
106188
Assertions.assertThat(flagSet.hasCapability(capability))
107-
.describedAs("Capabiilty of %s", capability)
189+
.describedAs("Capabiilty of %s on %s", capability, flagSet)
108190
.isTrue();
109191
}
110192

111-
private void assertCapabilityFalse(final String capability) {
193+
/**
194+
* Assert that the flagset lacks a capability.
195+
* @param capability capability to probe for
196+
*/
197+
private void assertLacksCapability(final String capability) {
112198
Assertions.assertThat(flagSet.hasCapability(capability))
113-
.describedAs("Capabiilty of %s", capability)
199+
.describedAs("Capabiilty of %s on %s", capability, flagSet)
114200
.isFalse();
115201
}
116202

117203

204+
/**
205+
* Test the * binding.
206+
*/
118207
@Test
119208
public void testFlagSetStarEntry() throws Throwable {
120209
final Configuration conf = mkConf("*");
121210
flagSet = buildFlagSet(SimpleEnum.class, conf, KEY, false);
122211
assertEnabled(SimpleEnum.a);
123212
assertEnabled(SimpleEnum.b);
124-
assertCapabilityTrue(KEY + ".a");
125-
assertCapabilityTrue(KEY + ".b");
213+
assertHasCapability(KEY + ".a");
214+
assertHasCapability(KEY + ".b");
126215
}
127216

128217
/**
129218
* Assert a flag is enabled.
130219
* @param flag flag to check
131220
*/
132221
private void assertEnabled(final SimpleEnum flag) {
133-
Assertions.assertThat(flagSet.enabled(flag)).describedAs("status of flag a").isTrue();
222+
Assertions.assertThat(flagSet.enabled(flag))
223+
.describedAs("status of flag %s in %s", flag, flagSet)
224+
225+
.isTrue();
134226
}
135227

136228
/**
137229
* Assert a flag is disabled.
138230
* @param flag flag to check
139231
*/
140232
private void assertDisabled(final SimpleEnum flag) {
141-
Assertions.assertThat(flagSet.enabled(flag)).describedAs("status of flag a").isFalse();
233+
Assertions.assertThat(flagSet.enabled(flag)).
234+
describedAs("status of flag %s in %s", flag, flagSet)
235+
.isFalse();
142236
}
143237
}

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -745,6 +745,8 @@ public void initialize(URI name, Configuration originalConf)
745745
boolean performanceCreation = conf.getBoolean(FS_S3A_CREATE_PERFORMANCE,
746746
performanceFlags.enabled(PerformanceFlagEnum.create));
747747
performanceFlags.set(PerformanceFlagEnum.create, performanceCreation);
748+
// freeze.
749+
performanceFlags.makeImmutable();
748750

749751
LOG.debug("{} = {}", FS_S3A_CREATE_PERFORMANCE, performanceCreation);
750752
allowAuthoritativePaths = S3Guard.getAuthoritativePaths(this);

0 commit comments

Comments
 (0)