001/**
002 * Copyright The Apache Software Foundation
003 *
004 * Licensed to the Apache Software Foundation (ASF) under one
005 * or more contributor license agreements.  See the NOTICE file
006 * distributed with this work for additional information
007 * regarding copyright ownership.  The ASF licenses this file
008 * to you under the Apache License, Version 2.0 (the
009 * "License"); you may not use this file except in compliance
010 * with the License.  You may obtain a copy of the License at
011 *
012 *     http://www.apache.org/licenses/LICENSE-2.0
013 *
014 * Unless required by applicable law or agreed to in writing, software
015 * distributed under the License is distributed on an "AS IS" BASIS,
016 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
017 * See the License for the specific language governing permissions and
018 * limitations under the License.
019 */
020package org.apache.hadoop.hbase;
021
022import java.io.DataOutput;
023import java.io.IOException;
024import java.io.OutputStream;
025import java.util.ArrayList;
026import java.util.HashMap;
027import java.util.Iterator;
028import java.util.List;
029import java.util.Map;
030
031import org.apache.hadoop.conf.Configuration;
032import org.apache.hadoop.hbase.util.Bytes;
033import org.apache.yetus.audience.InterfaceAudience;
034
035import org.apache.hbase.thirdparty.org.apache.commons.collections4.iterators.UnmodifiableIterator;
036
037/**
038 * Do a shallow merge of multiple KV configuration pools. This is a very useful
039 * utility class to easily add per-object configurations in addition to wider
040 * scope settings. This is different from Configuration.addResource()
041 * functionality, which performs a deep merge and mutates the common data
042 * structure.
043 * <p>
044 * The iterator on CompoundConfiguration is unmodifiable. Obtaining iterator is an expensive
045 * operation.
046 * <p>
047 * For clarity: the shallow merge allows the user to mutate either of the
048 * configuration objects and have changes reflected everywhere. In contrast to a
049 * deep merge, that requires you to explicitly know all applicable copies to
050 * propagate changes.
051 * 
052 * WARNING: The values set in the CompoundConfiguration are do not handle Property variable
053 * substitution.  However, if they are set in the underlying configuration substitutions are
054 * done. 
055 */
056@InterfaceAudience.Private
057public class CompoundConfiguration extends Configuration {
058
059  private Configuration mutableConf = null;
060
061  /**
062   * Default Constructor. Initializes empty configuration
063   */
064  public CompoundConfiguration() {
065  }
066
067  // Devs: these APIs are the same contract as their counterparts in
068  // Configuration.java
069  private interface ImmutableConfigMap extends Iterable<Map.Entry<String,String>> {
070    String get(String key);
071    String getRaw(String key);
072    Class<?> getClassByName(String name) throws ClassNotFoundException;
073    int size();
074  }
075
076  private final List<ImmutableConfigMap> configs = new ArrayList<>();
077
078  static class ImmutableConfWrapper implements  ImmutableConfigMap {
079   private final Configuration c;
080    
081    ImmutableConfWrapper(Configuration conf) {
082      c = conf;
083    }
084
085    @Override
086    public Iterator<Map.Entry<String,String>> iterator() {
087      return c.iterator();
088    }
089    
090    @Override
091    public String get(String key) {
092      return c.get(key);
093    }
094
095    @Override
096    public String getRaw(String key) {
097      return c.getRaw(key);
098    }
099
100    @Override
101    public Class<?> getClassByName(String name)
102        throws ClassNotFoundException {
103      return c.getClassByName(name);
104    }
105
106    @Override
107    public int size() {
108      return c.size();
109    }
110
111    @Override
112    public String toString() {
113      return c.toString();
114    }
115  }
116
117  /**
118   * If set has been called, it will create a mutableConf.  This converts the mutableConf to an
119   * immutable one and resets it to allow a new mutable conf.  This is used when a new map or
120   * conf is added to the compound configuration to preserve proper override semantics.
121   */
122  void freezeMutableConf() {
123    if (mutableConf == null) {
124      // do nothing if there is no current mutableConf
125      return;
126    }
127
128    this.configs.add(0, new ImmutableConfWrapper(mutableConf));
129    mutableConf = null;
130  }
131
132  /**
133   * Add Hadoop Configuration object to config list.
134   * The added configuration overrides the previous ones if there are name collisions.
135   * @param conf configuration object
136   * @return this, for builder pattern
137   */
138  public CompoundConfiguration add(final Configuration conf) {
139    freezeMutableConf();
140
141    if (conf instanceof CompoundConfiguration) {
142      this.configs.addAll(0, ((CompoundConfiguration) conf).configs);
143      return this;
144    }
145    // put new config at the front of the list (top priority)
146    this.configs.add(0, new ImmutableConfWrapper(conf));
147    return this;
148  }
149
150  /**
151   * Add Bytes map to config list. This map is generally
152   * created by HTableDescriptor or HColumnDescriptor, but can be abstractly
153   * used. The added configuration overrides the previous ones if there are
154   * name collisions.
155   *
156   * @param map
157   *          Bytes map
158   * @return this, for builder pattern
159   */
160  public CompoundConfiguration addBytesMap(
161      final Map<Bytes, Bytes> map) {
162    freezeMutableConf();
163
164    // put new map at the front of the list (top priority)
165    this.configs.add(0, new ImmutableConfigMap() {
166      private final Map<Bytes, Bytes> m = map;
167
168      @Override
169      public Iterator<Map.Entry<String,String>> iterator() {
170        Map<String, String> ret = new HashMap<>();
171        for (Map.Entry<Bytes, Bytes> entry : map.entrySet()) {
172          String key = Bytes.toString(entry.getKey().get());
173          String val = entry.getValue() == null ? null : Bytes.toString(entry.getValue().get());
174          ret.put(key, val);
175        }
176        return ret.entrySet().iterator();
177      }
178      
179      @Override
180      public String get(String key) {
181        Bytes ibw = new Bytes(Bytes
182            .toBytes(key));
183        if (!m.containsKey(ibw))
184          return null;
185        Bytes value = m.get(ibw);
186        if (value == null || value.get() == null)
187          return null;
188        return Bytes.toString(value.get());
189      }
190
191      @Override
192      public String getRaw(String key) {
193        return get(key);
194      }
195
196      @Override
197      public Class<?> getClassByName(String name)
198      throws ClassNotFoundException {
199        return null;
200      }
201
202      @Override
203      public int size() {
204        return m.size();
205      }
206
207      @Override
208      public String toString() {
209        return m.toString();
210      }
211    });
212    return this;
213  }
214
215  /**
216   * Add String map to config list. This map is generally created by HTableDescriptor
217   * or HColumnDescriptor, but can be abstractly used. The added configuration
218   * overrides the previous ones if there are name collisions.
219   *
220   * @return this, for builder pattern
221   */
222  public CompoundConfiguration addStringMap(final Map<String, String> map) {
223    freezeMutableConf();
224
225    // put new map at the front of the list (top priority)
226    this.configs.add(0, new ImmutableConfigMap() {
227      private final Map<String, String> m = map;
228
229      @Override
230      public Iterator<Map.Entry<String,String>> iterator() {
231        return map.entrySet().iterator();
232      }
233
234      @Override
235      public String get(String key) {
236        return m.get(key);
237      }
238
239      @Override
240      public String getRaw(String key) {
241        return get(key);
242      }
243
244      @Override
245      public Class<?> getClassByName(String name)
246      throws ClassNotFoundException {
247        return null;
248      }
249
250      @Override
251      public int size() {
252        return m.size();
253      }
254
255      @Override
256      public String toString() {
257        return m.toString();
258      }
259    });
260    return this;
261  }
262
263  @Override
264  public String toString() {
265    StringBuilder sb = new StringBuilder();
266    sb.append("CompoundConfiguration: " + this.configs.size() + " configs");
267    for (ImmutableConfigMap m : this.configs) {
268      sb.append(m);
269    }
270    return sb.toString();
271  }
272
273  @Override
274  public String get(String key) {
275    if (mutableConf != null) {
276      String value = mutableConf.get(key);
277      if (value != null) {
278        return value;
279      }
280    }
281
282    for (ImmutableConfigMap m : this.configs) {
283      String value = m.get(key);
284      if (value != null) {
285        return value;
286      }
287    }
288    return null;
289  }
290
291  @Override
292  public String getRaw(String key) {
293    if (mutableConf != null) {
294      String value = mutableConf.getRaw(key);
295      if (value != null) {
296        return value;
297      }
298    }
299
300    for (ImmutableConfigMap m : this.configs) {
301      String value = m.getRaw(key);
302      if (value != null) {
303        return value;
304      }
305    }
306    return null;
307  }
308
309  @Override
310  public Class<?> getClassByName(String name) throws ClassNotFoundException {
311    if (mutableConf != null) {
312      Class<?> value = mutableConf.getClassByName(name);
313      if (value != null) {
314        return value;
315      }
316    }
317
318    for (ImmutableConfigMap m : this.configs) {
319      Class<?> value = m.getClassByName(name);
320      if (value != null) {
321        return value;
322      }
323    }
324    throw new ClassNotFoundException();
325  }
326
327  // TODO: This method overestimates the number of configuration settings -- if a value is masked
328  // by an overriding config or map, it will be counted multiple times. 
329  @Override
330  public int size() {
331    int ret = 0;
332
333    if (mutableConf != null) {
334      ret += mutableConf.size();
335    }
336
337    for (ImmutableConfigMap m : this.configs) {
338      ret += m.size();
339    }
340    return ret;
341  }
342
343  /**
344   * Get the value of the <code>name</code>. If the key is deprecated,
345   * it returns the value of the first key which replaces the deprecated key
346   * and is not null.
347   * If no such property exists,
348   * then <code>defaultValue</code> is returned.
349
350   * The CompooundConfiguration does not do property substitution.  To do so we need
351   * Configuration.getProps to be protected or package visible.  Though in hadoop2 it is
352   * protected, in hadoop1 the method is private and not accessible.
353   * 
354   * All of the get* methods call this overridden get method.
355   * 
356   * @param name property name.
357   * @param defaultValue default value.
358   * @return property value, or <code>defaultValue</code> if the property 
359   *         doesn't exist.                    
360   **/
361  @Override
362  public String get(String name, String defaultValue) {
363    String ret = get(name);
364    return ret == null ? defaultValue : ret;
365  }
366
367  @Override
368  public Iterator<Map.Entry<String, String>> iterator() {
369    Map<String, String> ret = new HashMap<>();
370
371    // add in reverse order so that oldest get overridden.
372    if (!configs.isEmpty()) {
373      for (int i = configs.size() - 1; i >= 0; i--) {
374        ImmutableConfigMap map = configs.get(i);
375        Iterator<Map.Entry<String, String>> iter = map.iterator();
376        while (iter.hasNext()) {
377          Map.Entry<String, String> entry = iter.next();
378          ret.put(entry.getKey(), entry.getValue());
379        }
380      }
381    }
382
383    // add mutations to this CompoundConfiguration last.
384    if (mutableConf != null) {
385      Iterator<Map.Entry<String, String>> miter = mutableConf.iterator();
386      while (miter.hasNext()) {
387        Map.Entry<String, String> entry = miter.next();
388        ret.put(entry.getKey(), entry.getValue());
389      }
390    }
391
392    return UnmodifiableIterator.unmodifiableIterator(ret.entrySet().iterator());
393  }
394
395  @Override
396  public void set(String name, String value) {
397    if (mutableConf == null) {
398      // not thread safe
399      mutableConf = new Configuration(false); // an empty configuration
400    }
401    mutableConf.set(name,  value);
402  }
403
404  /***********************************************************************************************
405   * These methods are unsupported, and no code using CompoundConfiguration depend upon them.
406   * Quickly abort upon any attempts to use them. 
407   **********************************************************************************************/
408
409  @Override
410  public void clear() {
411    throw new UnsupportedOperationException("Immutable Configuration");
412  }
413
414  @Override
415  public void write(DataOutput out) throws IOException {
416    throw new UnsupportedOperationException("Immutable Configuration");
417  }
418
419  @Override
420  public void writeXml(OutputStream out) throws IOException {
421    throw new UnsupportedOperationException("Immutable Configuration");
422  }
423}