You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ke...@apache.org on 2013/12/04 21:12:10 UTC

[1/8] Generate thrift wrappers in the build process.

Updated Branches:
  refs/heads/master a2d7ee71a -> c72c92cf3


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/python/twitter/aurora/tools/java/thrift_wrapper_codegen.py
----------------------------------------------------------------------
diff --git a/src/main/python/twitter/aurora/tools/java/thrift_wrapper_codegen.py b/src/main/python/twitter/aurora/tools/java/thrift_wrapper_codegen.py
new file mode 100644
index 0000000..4376e85
--- /dev/null
+++ b/src/main/python/twitter/aurora/tools/java/thrift_wrapper_codegen.py
@@ -0,0 +1,475 @@
+#!/usr/bin/env python
+
+from __future__ import print_function
+
+import os
+import re
+import sys
+
+
+class Type(object):
+  '''A data type.'''
+
+  def __init__(self, name, package=None, immutable=False):
+    self.name = name
+    self.package = package
+    self.immutable = immutable
+
+  def absolute_name(self):
+    return '%s.%s' % (self.package, self.name) if self.package else self.name
+
+  def __str__(self):
+    return '%s (%smutable)' % (self.absolute_name(), 'im' if self.immutable else '')
+
+
+class PrimitiveType(Type):
+  '''A primitive type, with its associated typeboxed name.'''
+
+  def __init__(self, name, boxed_name):
+    Type.__init__(self, name, package=None, immutable=True)
+    self.boxed_name = boxed_name
+
+
+class ParameterizedType(Type):
+  '''A parameterized type, usually a collection.'''
+
+  def __init__(self, name, params):
+    Type.__init__(self, name, None)
+    self.params = params
+
+  def param_names(self):
+    def name(t):
+      if isinstance(t, StructType):
+        return t.codegen_name
+      elif isinstance(t, PrimitiveType):
+        return t.boxed_name
+      else:
+        return t.name
+    return ', '.join([name(p) for p in self.params])
+
+
+class StructType(Type):
+  '''A thrift-defined type, which composes other types as fields.'''
+
+  def __init__(self, name, package, kind, fields):
+    Type.__init__(self, name, package, kind == 'enum')
+    self.codegen_name = 'I%s' % name
+    self.kind = kind
+    self.fields = fields
+
+  def __str__(self):
+    return '%s %s { %s }' % (self.kind, self.name, ', '.join(map(str, self.fields)))
+
+
+class Field(object):
+  '''A field within a thrift structure.'''
+
+  def __init__(self, ttype, name):
+    self.ttype = ttype
+    self.name = name
+
+  def accessor_method(self):
+    return '%s%s' % (
+        'is' if self.ttype.name == 'boolean' else 'get',
+        self.name[:1].capitalize() + self.name[1:])
+
+  def isset_method(self):
+    return 'isSet%s' % (self.name[0].upper() + self.name[1:])
+
+  def __str__(self):
+    return '%s: %s' % (self.name, self.ttype)
+
+
+FIELD_TEMPLATE = '''  public %(type)s %(fn_name)s() {
+    return %(field)s;
+  }'''
+
+
+# Template string for a method to access an immutable field.
+IMMUTABLE_FIELD_TEMPLATE = '''  public %(type)s %(fn_name)s() {
+    return wrapped.%(fn_name)s();
+  }'''
+
+
+STRUCT_DECLARATION = '''private final %(type)s %(field)s;'''
+STRUCT_ASSIGNMENT = '''this.%(field)s = !wrapped.%(isset)s()
+        ? null
+        : %(type)s.buildNoCopy(wrapped.%(fn_name)s());'''
+
+
+IMMUTABLE_COLLECTION_DECLARATION = '''private final Immutable%(collection)s<%(params)s> %(field)s;'''
+IMMUTABLE_COLLECTION_ASSIGNMENT = '''this.%(field)s = !wrapped.%(isset)s()
+        ? Immutable%(collection)s.<%(params)s>of()
+        : Immutable%(collection)s.copyOf(wrapped.%(fn_name)s());'''
+
+
+# Tempalte string for assignment for a collection field containing a struct.
+STRUCT_COLLECTION_FIELD_ASSIGNMENT = '''this.%(field)s = !wrapped.%(isset)s()
+        ? Immutable%(collection)s.<%(params)s>of()
+        : FluentIterable.from(wrapped.%(fn_name)s())
+              .transform(%(params)s.FROM_BUILDER)
+              .to%(collection)s();'''
+
+PACKAGE_NAME = 'com.twitter.aurora.scheduler.storage.entities'
+
+CLASS_TEMPLATE = '''/*
+ * Copyright 2013 Twitter, 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 %(package)s;
+
+%(imports)s
+
+/**
+ * An immutable wrapper class.
+ * <p>
+ * This code is auto-generated, and should not be directly modified.
+ */
+public final class %(name)s {
+  private final %(wrapped)s wrapped;
+%(fields)s
+  private %(name)s(%(wrapped)s wrapped) {
+    this.wrapped = Preconditions.checkNotNull(wrapped);%(assignments)s
+  }
+
+  static %(name)s buildNoCopy(%(wrapped)s wrapped) {
+    return new %(name)s(wrapped);
+  }
+
+  public static %(name)s build(%(wrapped)s wrapped) {
+    return buildNoCopy(wrapped.deepCopy());
+  }
+
+  public static final Function<%(name)s, %(wrapped)s> TO_BUILDER =
+      new Function<%(name)s, %(wrapped)s>() {
+        @Override
+        public %(wrapped)s apply(%(name)s input) {
+          return input.newBuilder();
+        }
+      };
+
+  public static final Function<%(wrapped)s, %(name)s> FROM_BUILDER =
+      new Function<%(wrapped)s, %(name)s>() {
+        @Override
+        public %(name)s apply(%(wrapped)s input) {
+          return new %(name)s(input);
+        }
+      };
+
+  public static ImmutableList<%(wrapped)s> toBuildersList(Iterable<%(name)s> w) {
+    return FluentIterable.from(w).transform(TO_BUILDER).toList();
+  }
+
+  public static ImmutableList<%(name)s> listFromBuilders(Iterable<%(wrapped)s> b) {
+    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
+  }
+
+  public static ImmutableSet<%(wrapped)s> toBuildersSet(Iterable<%(name)s> w) {
+    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
+  }
+
+  public static ImmutableSet<%(name)s> setFromBuilders(Iterable<%(wrapped)s> b) {
+    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
+  }
+
+  public %(wrapped)s newBuilder() {
+    return wrapped.deepCopy();
+  }
+
+%(accessors)s
+
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof %(name)s)) {
+      return false;
+    }
+    %(name)s other = (%(name)s) o;
+    return wrapped.equals(other.wrapped);
+  }
+
+  @Override
+  public int hashCode() {
+    return wrapped.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return wrapped.toString();
+  }
+}'''
+
+
+class GeneratedCode(object):
+  def __init__(self, class_name, wrapped_type):
+    self._class_name = class_name
+    self._wrapped_type = wrapped_type
+    self._imports = set()
+    self._accessors = []
+    self._fields = []
+    self._assignments = []
+
+  def add(self, s, end='\n'):
+    print('This no longer does anything.')
+
+  def add_import(self, import_class):
+    self._imports.add(import_class)
+
+  def add_assignment(self, field, assignment):
+    self._fields.append(field)
+    self._assignments.append(assignment)
+
+  def add_accessor(self, accessor_method):
+    self._accessors.append(accessor_method)
+
+  def dump(self, f):
+    remaining_imports = list(self._imports)
+    import_groups = []
+    def remove_by_prefix(prefix):
+      group = [i for i in remaining_imports if i.startswith(prefix)]
+      remaining_imports[:] = [i for i in remaining_imports if not i.startswith(prefix)]
+      return group
+
+    def add_import_group(group):
+      if group:
+        import_groups.append('\n'.join(['import %s;' % i for i in sorted(group)]))
+
+    twitter_imports = remove_by_prefix('com.twitter')
+    add_import_group(remove_by_prefix('java'))
+    add_import_group(remove_by_prefix('com'))
+    add_import_group(remove_by_prefix('net'))
+    add_import_group(remove_by_prefix('org'))
+    add_import_group(twitter_imports)
+
+    print(CLASS_TEMPLATE % {
+      'package': PACKAGE_NAME,
+      'name': self._class_name,
+      'wrapped': self._wrapped_type,
+      'imports': '\n\n'.join(import_groups),
+      'accessors': '\n\n'.join(self._accessors),
+      'fields':  ('  ' + '\n  '.join(self._fields) + '\n') if self._fields else '',
+      'assignments': ('\n    ' + '\n    '.join(self._assignments)) if self._assignments else '',
+    }, file=f)
+
+
+# A namespace declaration, e.g.:
+#    namespace java com.twitter.aurora.gen
+NAMESPACE_RE = 'namespace\s+(?P<lang>\w+)\s+(?P<namespace>[^\s]+)'
+
+# A possibly-parameterized type name, e.g.:
+#    int
+#    TaskConfig
+#    Set<String>
+#    Map<String, TaskConfig>
+TYPE_PATTERN = '(?P<type>\w+)(?:<(?P<params>[^>]+)>)?'
+
+
+# Matches a complete struct definnition, capturing the type and body.
+STRUCT_RE = '(?P<kind>enum|struct|union)\s+(?P<name>\w+)\s+{(?P<body>[^}]+)}'
+
+
+# A field definition within a struct, e.g.:
+#     1: string name
+#     15: Map<String, TaskConfig> configs  # Configs mapped by name.
+FIELD_RE = '\s*\d+:\s+(?:(?:required|optional)\s+)?(%s)\s+(?P<name>\w+).*' % TYPE_PATTERN
+
+
+def parse_structs(thrift_defs):
+  '''Read all thrift structures found in a file.
+
+  This returns a list of Type objects representing the structs found
+  and the fields they contain.
+  '''
+  # Capture all namespace definitions.
+  namespaces = dict(re.findall(NAMESPACE_RE, thrift_defs))
+
+  def parse_field(field):
+    type_name = field.group('type')
+    type_params = field.group('params')
+    if type_params:
+      params = [Type(p) for p in type_params.replace(' ', '').split(',')]
+      ttype = ParameterizedType(type_name, params)
+    else:
+      ttype = Type(type_name)
+    return Field(ttype, field.group('name'))
+
+  def parse_fields(field_str):
+    return map(parse_field, re.finditer(FIELD_RE, field_str))
+
+  return [StructType(s.group('name'),
+                     namespaces['java'],
+                     s.group('kind'),
+                     parse_fields(s.group('body')))
+          for s in re.finditer(STRUCT_RE, thrift_defs, flags=re.MULTILINE)]
+
+
+def generate_java(struct):
+  code = GeneratedCode(struct.codegen_name, struct.name)
+  code.add_import('com.google.common.base.Preconditions')
+  code.add_import('com.google.common.base.Function')
+  code.add_import('com.google.common.collect.ImmutableList')
+  code.add_import('com.google.common.collect.ImmutableSet')
+  code.add_import('com.google.common.collect.FluentIterable')
+  code.add_import(struct.absolute_name())
+
+  if struct.kind == 'union':
+    code.add_accessor(IMMUTABLE_FIELD_TEMPLATE
+                      % {'type': '%s._Fields' % struct.name, 'fn_name': 'getSetField'})
+
+  # Accessor for each field.
+  for field in struct.fields:
+    if not (isinstance(field.ttype, StructType) and (field.ttype.kind == 'enum' or struct.kind == 'union')):
+      code.add_accessor(IMMUTABLE_FIELD_TEMPLATE
+                        % {'type': 'boolean',
+                           'fn_name': field.isset_method()})
+    if field.ttype.immutable:
+      code.add_accessor(IMMUTABLE_FIELD_TEMPLATE % {'type': field.ttype.name,
+                                                    'fn_name': field.accessor_method()})
+    elif not struct.kind == 'union':
+      if isinstance(field.ttype, StructType):
+        return_type = field.ttype.codegen_name
+      elif isinstance(field.ttype, ParameterizedType):
+        return_type = '%s<%s>' % (field.ttype.name, field.ttype.param_names())
+      else:
+        return_type = field.ttype.name
+      code.add_accessor(FIELD_TEMPLATE % {'type': return_type,
+                                          'fn_name': field.accessor_method(),
+                                          'field': field.name})
+
+    if isinstance(field.ttype, StructType):
+      if field.ttype.kind == 'enum':
+        code.add_import(field.ttype.absolute_name())
+
+      if field.ttype.immutable:
+        # Direct accessor was already added.
+        pass
+      elif struct.kind == 'union':
+        copy_field = '%s.build(wrapped.%s())' % (field.ttype.codegen_name,
+                                                 field.accessor_method())
+        code.add_accessor(FIELD_TEMPLATE % {'type': field.ttype.codegen_name,
+                                            'fn_name': field.accessor_method(),
+                                            'field': copy_field})
+      else:
+        args = {
+          'field': field.name,
+          'fn_name': field.accessor_method(),
+          'isset': field.isset_method(),
+          'type': field.ttype.codegen_name,
+        }
+        code.add_assignment(STRUCT_DECLARATION % args, STRUCT_ASSIGNMENT % args)
+    elif isinstance(field.ttype, ParameterizedType):
+      # Add necessary imports, supporting only List, Map, Set.
+      assert field.ttype.name in ['List', 'Map', 'Set'], 'Unrecognized type %s' % field.ttype.name
+      code.add_import('com.google.common.collect.Immutable%s' % field.ttype.name)
+      code.add_import('java.util.%s' % field.ttype.name)
+
+      params = field.ttype.params
+      if all([p.immutable for p in params]):
+        # All parameter types are immutable.
+        assignment = IMMUTABLE_COLLECTION_ASSIGNMENT
+      elif len(params) == 1:
+        # Only one non-immutable parameter.
+        # Assumes the parameter type is a struct and our code generator
+        # will make a compatible wrapper class and constructor.
+        assignment = STRUCT_COLLECTION_FIELD_ASSIGNMENT
+      else:
+        assert False, 'Unable to codegen accessor field for %s' % field.name
+      args = {'collection': field.ttype.name,
+              'field': field.name,
+              'fn_name': field.accessor_method(),
+              'isset': field.isset_method(),
+              'params': field.ttype.param_names()}
+      code.add_assignment(IMMUTABLE_COLLECTION_DECLARATION % args, assignment % args)
+    elif not field.ttype.immutable:
+      assert False, 'Making type %s immutable is not supported.' % field.ttype.name
+  return code
+
+
+THRIFT_ALIASES = {
+  'bool': 'boolean',
+  'i32': 'int',
+  'i64': 'long',
+  'double': 'double',
+  'string': 'String',
+  'list': 'List',
+  'set': 'Set',
+  'map': 'Map',
+  'binary': 'byte[]',
+}
+
+
+def main(args):
+  if len(args) != 4:
+    print('usage: %s thrift_file struct_name output_directory' % sys.argv[0])
+    sys.exit(1)
+
+  thrift_file, struct_name, output_directory = sys.argv[1:]
+  print('Searching for %s in %s' % (sys.argv[2], sys.argv[1]))
+  with open(sys.argv[1]) as f:
+    # Load all structs found in the thrift file.
+    structs = parse_structs(f.read())
+
+    # The symbol table stores information about types we recognize.
+    # As new symbols are parsed, they are accumulated here.
+    # This is also seeded with JDK types.
+    # Note: byte[] is not immutable, but we'd rather accept it than copy.
+    primitives = dict((t, PrimitiveType(t, b)) for (t, b) in [('boolean', 'Boolean'),
+                                                             ('int', 'Integer'),
+                                                             ('long', 'Long'),
+                                                             ('double', 'Double')])
+    lang_symbols = dict((t, Type(t, 'java.lang', immutable=True)) for t in ['String', 'byte[]'])
+    util_symbols = dict((t, Type(t, 'java.util')) for t in ['List', 'Map', 'Set'])
+    symbol_table = dict(primitives.items() + lang_symbols.items() + util_symbols.items())
+
+    def load_dependencies(struct):
+      # Fill in type information for fields by searching for dependencies.
+      for field in struct.fields:
+        if isinstance(field.ttype, ParameterizedType):
+          field.ttype.name = find_symbol(field.ttype.name).name
+          field.ttype.params = [find_symbol(p.name) for p in field.ttype.params]
+        else:
+          field.ttype = find_symbol(field.ttype.name)
+
+    def find_symbol(name):
+      name = THRIFT_ALIASES.get(name, name)
+      if name in symbol_table:
+        return symbol_table[name]
+
+      symbol = next((s for s in structs if s.name == name), None)
+      assert symbol, 'Failed to find required struct %s' % name
+      load_dependencies(symbol)
+      symbol_table[name] = symbol
+      return symbol
+
+    find_symbol(sys.argv[2])
+    print('Symbol table:')
+    for _, symbol in symbol_table.items():
+      print('    %s' % symbol)
+
+    for _, symbol in symbol_table.items():
+      if isinstance(symbol, StructType):
+        if symbol.kind == 'enum':
+          print('Skipping code generation for %s, since it is immutable' % symbol.name)
+        else:
+          package_dir = os.path.join(sys.argv[3], PACKAGE_NAME.replace('.', os.path.sep))
+          if not os.path.isdir(package_dir):
+            os.makedirs(package_dir)
+          gen_file = os.path.join(package_dir, '%s.java' % symbol.codegen_name)
+          print('Generating %s' % gen_file)
+          with open(gen_file, 'w') as f:
+            code = generate_java(symbol)
+            code.dump(f)
+
+
+if __name__ == '__main__':
+  main(sys.argv)


[5/8] git commit: Adding generated files into jar

Posted by ke...@apache.org.
Adding generated files into jar


Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/ce669c1a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/ce669c1a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/ce669c1a

Branch: refs/heads/master
Commit: ce669c1aa277272441cd3b94fd59c6144834d829
Parents: 2f79ec3
Author: Maxim Khutornenko <mk...@twitter.com>
Authored: Tue Nov 26 17:51:02 2013 -0800
Committer: Maxim Khutornenko <mk...@twitter.com>
Committed: Tue Nov 26 17:51:02 2013 -0800

----------------------------------------------------------------------
 build.gradle | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/ce669c1a/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index d5e08ab..b39d837 100644
--- a/build.gradle
+++ b/build.gradle
@@ -79,6 +79,10 @@ sourceSets {
   }
 }
 
+jar {
+    from sourceSets.generated.output
+}
+
 dependencies {
   compile 'aopalliance:aopalliance:1.0'
   compile 'com.google.code.findbugs:jsr305:1.3.9'


[3/8] git commit: Clean up codegen refs in build.gradle.

Posted by ke...@apache.org.
Clean up codegen refs in build.gradle.


Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/29d39b1d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/29d39b1d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/29d39b1d

Branch: refs/heads/master
Commit: 29d39b1d479f3f82a3857f103e67cf5104105b9d
Parents: 8704736
Author: Bill Farner <bi...@twitter.com>
Authored: Tue Nov 26 13:20:24 2013 -0800
Committer: Bill Farner <bi...@twitter.com>
Committed: Tue Nov 26 13:20:24 2013 -0800

----------------------------------------------------------------------
 build.gradle | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/29d39b1d/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 44d6cbf..d5e08ab 100644
--- a/build.gradle
+++ b/build.gradle
@@ -6,6 +6,7 @@ apply plugin: 'maven-publish'
 
 buildDir = 'dist'
 def generatedDir = "$buildDir/generated-src"
+def generatedJavaDir = "$generatedDir/gen-java"
 
 compileJava {
   sourceCompatibility = 1.7
@@ -178,6 +179,8 @@ task generateSources(dependsOn: 'bootstrapThrift') {
         args = ['--gen', 'java:hashcode', '-o', outputDir, file]
       }
     }
+    // These are the 'root' structs, the tool recursively generates all composed structs.
+    // TODO(wfarner): Change codegen tool to generate for all structs in a thrift file.
     ['JobConfiguration', 'Lock', 'ScheduledTask', 'Quota'].each {
       def structName = it
       exec {
@@ -185,7 +188,7 @@ task generateSources(dependsOn: 'bootstrapThrift') {
         args = ['src/main/python/twitter/aurora/tools/java/thrift_wrapper_codegen.py',
                 'src/main/thrift/com/twitter/aurora/gen/api.thrift',
                 structName,
-                "$generatedDir/gen-java"]
+                generatedJavaDir]
       }
     }
   }
@@ -208,7 +211,7 @@ idea {
     iml {
       beforeMerged { module ->
         module.sourceFolders.add(new org.gradle.plugins.ide.idea.model.Path(
-            'file://$MODULE_DIR$/dist/generated-src/gen-java'))
+            "file://$generatedJavaDir"))
       }
       whenMerged { module ->
         // TODO(zmanji): Find a way to remove the `dist` directory instead of


[7/8] git commit: Add troubleshooting section to vagrant.md

Posted by ke...@apache.org.
Add troubleshooting section to vagrant.md

Add a troubleshooting section to the vagrant documentation.


Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/130d4668
Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/130d4668
Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/130d4668

Branch: refs/heads/master
Commit: 130d466852800643410f1f55c8e3cf34fc4a36ab
Parents: 52b566c
Author: Zameer Manji <zm...@twitter.com>
Authored: Mon Dec 2 18:06:50 2013 -0800
Committer: Zameer Manji <zm...@twitter.com>
Committed: Mon Dec 2 18:06:50 2013 -0800

----------------------------------------------------------------------
 docs/vagrant.md | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/130d4668/docs/vagrant.md
----------------------------------------------------------------------
diff --git a/docs/vagrant.md b/docs/vagrant.md
index 971a662..15d72bc 100644
--- a/docs/vagrant.md
+++ b/docs/vagrant.md
@@ -9,3 +9,10 @@ The observer is listening on http://192.168.33.4:1338/
 The master is listening on http://192.168.33.3:5050/
 
 Once everything is up, you can `vagrant ssh aurora-scheduler` and execute aurora client commands using the `aurora` client.
+
+Troubleshooting
+---------------
+Most of the vagrant related problems can be fixed by the following steps:
+* Destroying the vagrant environment with `vagrant destroy`
+* Cleaning the repository of build artifacts and other intermediate output with `git clean -fdx`
+* Bringing up the vagrant environment with `vagrant up`


[2/8] git commit: Generate thrift wrappers in the build process.

Posted by ke...@apache.org.
Generate thrift wrappers in the build process.


Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/87047365
Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/87047365
Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/87047365

Branch: refs/heads/master
Commit: 87047365bbf6619677051c8b8485d1e93614fbba
Parents: a2d7ee7
Author: Bill Farner <bi...@twitter.com>
Authored: Tue Nov 26 12:56:29 2013 -0800
Committer: Bill Farner <bi...@twitter.com>
Committed: Tue Nov 26 12:56:29 2013 -0800

----------------------------------------------------------------------
 build.gradle                                    |  19 +-
 .../storage/entities/IAssignedTask.java         | 161 -------
 .../scheduler/storage/entities/IConstraint.java | 122 -----
 .../storage/entities/IExecutorConfig.java       | 118 -----
 .../scheduler/storage/entities/IIdentity.java   | 118 -----
 .../storage/entities/IJobConfiguration.java     | 159 -------
 .../scheduler/storage/entities/IJobKey.java     | 126 -----
 .../storage/entities/ILimitConstraint.java      | 110 -----
 .../scheduler/storage/entities/ILock.java       | 146 ------
 .../scheduler/storage/entities/ILockKey.java    | 110 -----
 .../scheduler/storage/entities/IPackage.java    | 126 -----
 .../scheduler/storage/entities/IQuota.java      | 126 -----
 .../storage/entities/IScheduledTask.java        | 151 ------
 .../scheduler/storage/entities/ITaskConfig.java | 270 -----------
 .../storage/entities/ITaskConstraint.java       | 114 -----
 .../scheduler/storage/entities/ITaskEvent.java  | 131 -----
 .../storage/entities/IValueConstraint.java      | 124 -----
 .../aurora/tools/java/organize_imports.py       | 130 +++++
 .../aurora/tools/java/thrift_wrapper_codegen.py | 475 +++++++++++++++++++
 19 files changed, 621 insertions(+), 2215 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 6ba6da9..44d6cbf 100644
--- a/build.gradle
+++ b/build.gradle
@@ -81,7 +81,8 @@ sourceSets {
 dependencies {
   compile 'aopalliance:aopalliance:1.0'
   compile 'com.google.code.findbugs:jsr305:1.3.9'
-  compile 'com.google.guava:guava:14.0.1'
+  def guavaDep = 'com.google.guava:guava:14.0.1'
+  compile guavaDep
   compile 'com.google.inject:guice:3.0'
   compile 'com.google.protobuf:protobuf-java:2.4.1'
   def jerseyRev = '1.12'
@@ -142,6 +143,7 @@ dependencies {
   compile 'com.twitter.common:zookeeper-testing:0.0.37'
   testCompile 'junit:junit:4.10'
 
+  generatedCompile guavaDep
   generatedCompile thriftLib
 
   checkstyle 'com.puppycrawl.tools:checkstyle:5.6'
@@ -176,11 +178,21 @@ task generateSources(dependsOn: 'bootstrapThrift') {
         args = ['--gen', 'java:hashcode', '-o', outputDir, file]
       }
     }
+    ['JobConfiguration', 'Lock', 'ScheduledTask', 'Quota'].each {
+      def structName = it
+      exec {
+        executable = 'python'
+        args = ['src/main/python/twitter/aurora/tools/java/thrift_wrapper_codegen.py',
+                'src/main/thrift/com/twitter/aurora/gen/api.thrift',
+                structName,
+                "$generatedDir/gen-java"]
+      }
+    }
   }
 }
 
 compileGeneratedJava {
-  dependsOn(generateSources)
+  dependsOn generateSources
 }
 
 compileJava.source sourceSets.main.java
@@ -195,7 +207,8 @@ idea {
    module {
     iml {
       beforeMerged { module ->
-        module.sourceFolders.add(new org.gradle.plugins.ide.idea.model.Path('file://$MODULE_DIR$/dist/generated-src/gen-java'))
+        module.sourceFolders.add(new org.gradle.plugins.ide.idea.model.Path(
+            'file://$MODULE_DIR$/dist/generated-src/gen-java'))
       }
       whenMerged { module ->
         // TODO(zmanji): Find a way to remove the `dist` directory instead of

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/IAssignedTask.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IAssignedTask.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/IAssignedTask.java
deleted file mode 100644
index c73d128..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IAssignedTask.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import java.util.Map;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.AssignedTask;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class IAssignedTask {
-  private final AssignedTask wrapped;
-  private final ITaskConfig task;
-  private final ImmutableMap<String, Integer> assignedPorts;
-
-  private IAssignedTask(AssignedTask wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-    this.task = !wrapped.isSetTask()
-        ? null
-        : ITaskConfig.buildNoCopy(wrapped.getTask());
-    this.assignedPorts = !wrapped.isSetAssignedPorts()
-        ? ImmutableMap.<String, Integer>of()
-        : ImmutableMap.copyOf(wrapped.getAssignedPorts());
-  }
-
-  static IAssignedTask buildNoCopy(AssignedTask wrapped) {
-    return new IAssignedTask(wrapped);
-  }
-
-  public static IAssignedTask build(AssignedTask wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<IAssignedTask, AssignedTask> TO_BUILDER =
-      new Function<IAssignedTask, AssignedTask>() {
-        @Override
-        public AssignedTask apply(IAssignedTask input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<AssignedTask, IAssignedTask> FROM_BUILDER =
-      new Function<AssignedTask, IAssignedTask>() {
-        @Override
-        public IAssignedTask apply(AssignedTask input) {
-          return new IAssignedTask(input);
-        }
-      };
-
-  public static ImmutableList<AssignedTask> toBuildersList(Iterable<IAssignedTask> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<IAssignedTask> listFromBuilders(Iterable<AssignedTask> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<AssignedTask> toBuildersSet(Iterable<IAssignedTask> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<IAssignedTask> setFromBuilders(Iterable<AssignedTask> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public AssignedTask newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetTaskId() {
-    return wrapped.isSetTaskId();
-  }
-
-  public String getTaskId() {
-    return wrapped.getTaskId();
-  }
-
-  public boolean isSetSlaveId() {
-    return wrapped.isSetSlaveId();
-  }
-
-  public String getSlaveId() {
-    return wrapped.getSlaveId();
-  }
-
-  public boolean isSetSlaveHost() {
-    return wrapped.isSetSlaveHost();
-  }
-
-  public String getSlaveHost() {
-    return wrapped.getSlaveHost();
-  }
-
-  public boolean isSetTask() {
-    return wrapped.isSetTask();
-  }
-
-  public ITaskConfig getTask() {
-    return task;
-  }
-
-  public boolean isSetAssignedPorts() {
-    return wrapped.isSetAssignedPorts();
-  }
-
-  public Map<String, Integer> getAssignedPorts() {
-    return assignedPorts;
-  }
-
-  public boolean isSetInstanceId() {
-    return wrapped.isSetInstanceId();
-  }
-
-  public int getInstanceId() {
-    return wrapped.getInstanceId();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof IAssignedTask)) {
-      return false;
-    }
-    IAssignedTask other = (IAssignedTask) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/IConstraint.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IConstraint.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/IConstraint.java
deleted file mode 100644
index 8a4101d..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IConstraint.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.Constraint;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class IConstraint {
-  private final Constraint wrapped;
-  private final ITaskConstraint constraint;
-
-  private IConstraint(Constraint wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-    this.constraint = !wrapped.isSetConstraint()
-        ? null
-        : ITaskConstraint.buildNoCopy(wrapped.getConstraint());
-  }
-
-  static IConstraint buildNoCopy(Constraint wrapped) {
-    return new IConstraint(wrapped);
-  }
-
-  public static IConstraint build(Constraint wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<IConstraint, Constraint> TO_BUILDER =
-      new Function<IConstraint, Constraint>() {
-        @Override
-        public Constraint apply(IConstraint input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<Constraint, IConstraint> FROM_BUILDER =
-      new Function<Constraint, IConstraint>() {
-        @Override
-        public IConstraint apply(Constraint input) {
-          return new IConstraint(input);
-        }
-      };
-
-  public static ImmutableList<Constraint> toBuildersList(Iterable<IConstraint> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<IConstraint> listFromBuilders(Iterable<Constraint> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<Constraint> toBuildersSet(Iterable<IConstraint> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<IConstraint> setFromBuilders(Iterable<Constraint> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public Constraint newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetName() {
-    return wrapped.isSetName();
-  }
-
-  public String getName() {
-    return wrapped.getName();
-  }
-
-  public boolean isSetConstraint() {
-    return wrapped.isSetConstraint();
-  }
-
-  public ITaskConstraint getConstraint() {
-    return constraint;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof IConstraint)) {
-      return false;
-    }
-    IConstraint other = (IConstraint) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/IExecutorConfig.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IExecutorConfig.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/IExecutorConfig.java
deleted file mode 100644
index e7e6de0..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IExecutorConfig.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.ExecutorConfig;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class IExecutorConfig {
-  private final ExecutorConfig wrapped;
-
-  private IExecutorConfig(ExecutorConfig wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-  }
-
-  static IExecutorConfig buildNoCopy(ExecutorConfig wrapped) {
-    return new IExecutorConfig(wrapped);
-  }
-
-  public static IExecutorConfig build(ExecutorConfig wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<IExecutorConfig, ExecutorConfig> TO_BUILDER =
-      new Function<IExecutorConfig, ExecutorConfig>() {
-        @Override
-        public ExecutorConfig apply(IExecutorConfig input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<ExecutorConfig, IExecutorConfig> FROM_BUILDER =
-      new Function<ExecutorConfig, IExecutorConfig>() {
-        @Override
-        public IExecutorConfig apply(ExecutorConfig input) {
-          return new IExecutorConfig(input);
-        }
-      };
-
-  public static ImmutableList<ExecutorConfig> toBuildersList(Iterable<IExecutorConfig> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<IExecutorConfig> listFromBuilders(Iterable<ExecutorConfig> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<ExecutorConfig> toBuildersSet(Iterable<IExecutorConfig> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<IExecutorConfig> setFromBuilders(Iterable<ExecutorConfig> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public ExecutorConfig newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetName() {
-    return wrapped.isSetName();
-  }
-
-  public String getName() {
-    return wrapped.getName();
-  }
-
-  public boolean isSetData() {
-    return wrapped.isSetData();
-  }
-
-  public String getData() {
-    return wrapped.getData();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof IExecutorConfig)) {
-      return false;
-    }
-    IExecutorConfig other = (IExecutorConfig) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/IIdentity.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IIdentity.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/IIdentity.java
deleted file mode 100644
index 5e33ff3..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IIdentity.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.Identity;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class IIdentity {
-  private final Identity wrapped;
-
-  private IIdentity(Identity wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-  }
-
-  static IIdentity buildNoCopy(Identity wrapped) {
-    return new IIdentity(wrapped);
-  }
-
-  public static IIdentity build(Identity wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<IIdentity, Identity> TO_BUILDER =
-      new Function<IIdentity, Identity>() {
-        @Override
-        public Identity apply(IIdentity input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<Identity, IIdentity> FROM_BUILDER =
-      new Function<Identity, IIdentity>() {
-        @Override
-        public IIdentity apply(Identity input) {
-          return new IIdentity(input);
-        }
-      };
-
-  public static ImmutableList<Identity> toBuildersList(Iterable<IIdentity> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<IIdentity> listFromBuilders(Iterable<Identity> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<Identity> toBuildersSet(Iterable<IIdentity> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<IIdentity> setFromBuilders(Iterable<Identity> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public Identity newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetRole() {
-    return wrapped.isSetRole();
-  }
-
-  public String getRole() {
-    return wrapped.getRole();
-  }
-
-  public boolean isSetUser() {
-    return wrapped.isSetUser();
-  }
-
-  public String getUser() {
-    return wrapped.getUser();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof IIdentity)) {
-      return false;
-    }
-    IIdentity other = (IIdentity) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/IJobConfiguration.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IJobConfiguration.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/IJobConfiguration.java
deleted file mode 100644
index f698645..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IJobConfiguration.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.CronCollisionPolicy;
-import com.twitter.aurora.gen.JobConfiguration;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class IJobConfiguration {
-  private final JobConfiguration wrapped;
-  private final IJobKey key;
-  private final IIdentity owner;
-  private final ITaskConfig taskConfig;
-
-  private IJobConfiguration(JobConfiguration wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-    this.key = !wrapped.isSetKey()
-        ? null
-        : IJobKey.buildNoCopy(wrapped.getKey());
-    this.owner = !wrapped.isSetOwner()
-        ? null
-        : IIdentity.buildNoCopy(wrapped.getOwner());
-    this.taskConfig = !wrapped.isSetTaskConfig()
-        ? null
-        : ITaskConfig.buildNoCopy(wrapped.getTaskConfig());
-  }
-
-  static IJobConfiguration buildNoCopy(JobConfiguration wrapped) {
-    return new IJobConfiguration(wrapped);
-  }
-
-  public static IJobConfiguration build(JobConfiguration wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<IJobConfiguration, JobConfiguration> TO_BUILDER =
-      new Function<IJobConfiguration, JobConfiguration>() {
-        @Override
-        public JobConfiguration apply(IJobConfiguration input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<JobConfiguration, IJobConfiguration> FROM_BUILDER =
-      new Function<JobConfiguration, IJobConfiguration>() {
-        @Override
-        public IJobConfiguration apply(JobConfiguration input) {
-          return new IJobConfiguration(input);
-        }
-      };
-
-  public static ImmutableList<JobConfiguration> toBuildersList(Iterable<IJobConfiguration> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<IJobConfiguration> listFromBuilders(Iterable<JobConfiguration> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<JobConfiguration> toBuildersSet(Iterable<IJobConfiguration> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<IJobConfiguration> setFromBuilders(Iterable<JobConfiguration> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public JobConfiguration newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetKey() {
-    return wrapped.isSetKey();
-  }
-
-  public IJobKey getKey() {
-    return key;
-  }
-
-  public boolean isSetOwner() {
-    return wrapped.isSetOwner();
-  }
-
-  public IIdentity getOwner() {
-    return owner;
-  }
-
-  public boolean isSetCronSchedule() {
-    return wrapped.isSetCronSchedule();
-  }
-
-  public String getCronSchedule() {
-    return wrapped.getCronSchedule();
-  }
-
-  public CronCollisionPolicy getCronCollisionPolicy() {
-    return wrapped.getCronCollisionPolicy();
-  }
-
-  public boolean isSetTaskConfig() {
-    return wrapped.isSetTaskConfig();
-  }
-
-  public ITaskConfig getTaskConfig() {
-    return taskConfig;
-  }
-
-  public boolean isSetInstanceCount() {
-    return wrapped.isSetInstanceCount();
-  }
-
-  public int getInstanceCount() {
-    return wrapped.getInstanceCount();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof IJobConfiguration)) {
-      return false;
-    }
-    IJobConfiguration other = (IJobConfiguration) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/IJobKey.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IJobKey.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/IJobKey.java
deleted file mode 100644
index ff08d75..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IJobKey.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.JobKey;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class IJobKey {
-  private final JobKey wrapped;
-
-  private IJobKey(JobKey wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-  }
-
-  static IJobKey buildNoCopy(JobKey wrapped) {
-    return new IJobKey(wrapped);
-  }
-
-  public static IJobKey build(JobKey wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<IJobKey, JobKey> TO_BUILDER =
-      new Function<IJobKey, JobKey>() {
-        @Override
-        public JobKey apply(IJobKey input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<JobKey, IJobKey> FROM_BUILDER =
-      new Function<JobKey, IJobKey>() {
-        @Override
-        public IJobKey apply(JobKey input) {
-          return new IJobKey(input);
-        }
-      };
-
-  public static ImmutableList<JobKey> toBuildersList(Iterable<IJobKey> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<IJobKey> listFromBuilders(Iterable<JobKey> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<JobKey> toBuildersSet(Iterable<IJobKey> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<IJobKey> setFromBuilders(Iterable<JobKey> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public JobKey newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetRole() {
-    return wrapped.isSetRole();
-  }
-
-  public String getRole() {
-    return wrapped.getRole();
-  }
-
-  public boolean isSetEnvironment() {
-    return wrapped.isSetEnvironment();
-  }
-
-  public String getEnvironment() {
-    return wrapped.getEnvironment();
-  }
-
-  public boolean isSetName() {
-    return wrapped.isSetName();
-  }
-
-  public String getName() {
-    return wrapped.getName();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof IJobKey)) {
-      return false;
-    }
-    IJobKey other = (IJobKey) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILimitConstraint.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILimitConstraint.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILimitConstraint.java
deleted file mode 100644
index 2843794..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILimitConstraint.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.LimitConstraint;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class ILimitConstraint {
-  private final LimitConstraint wrapped;
-
-  private ILimitConstraint(LimitConstraint wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-  }
-
-  static ILimitConstraint buildNoCopy(LimitConstraint wrapped) {
-    return new ILimitConstraint(wrapped);
-  }
-
-  public static ILimitConstraint build(LimitConstraint wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<ILimitConstraint, LimitConstraint> TO_BUILDER =
-      new Function<ILimitConstraint, LimitConstraint>() {
-        @Override
-        public LimitConstraint apply(ILimitConstraint input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<LimitConstraint, ILimitConstraint> FROM_BUILDER =
-      new Function<LimitConstraint, ILimitConstraint>() {
-        @Override
-        public ILimitConstraint apply(LimitConstraint input) {
-          return new ILimitConstraint(input);
-        }
-      };
-
-  public static ImmutableList<LimitConstraint> toBuildersList(Iterable<ILimitConstraint> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<ILimitConstraint> listFromBuilders(Iterable<LimitConstraint> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<LimitConstraint> toBuildersSet(Iterable<ILimitConstraint> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<ILimitConstraint> setFromBuilders(Iterable<LimitConstraint> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public LimitConstraint newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetLimit() {
-    return wrapped.isSetLimit();
-  }
-
-  public int getLimit() {
-    return wrapped.getLimit();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof ILimitConstraint)) {
-      return false;
-    }
-    ILimitConstraint other = (ILimitConstraint) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILock.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILock.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILock.java
deleted file mode 100644
index 829458f..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILock.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.Lock;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class ILock {
-  private final Lock wrapped;
-  private final ILockKey key;
-
-  private ILock(Lock wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-    this.key = !wrapped.isSetKey()
-        ? null
-        : ILockKey.buildNoCopy(wrapped.getKey());
-  }
-
-  static ILock buildNoCopy(Lock wrapped) {
-    return new ILock(wrapped);
-  }
-
-  public static ILock build(Lock wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<ILock, Lock> TO_BUILDER =
-      new Function<ILock, Lock>() {
-        @Override
-        public Lock apply(ILock input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<Lock, ILock> FROM_BUILDER =
-      new Function<Lock, ILock>() {
-        @Override
-        public ILock apply(Lock input) {
-          return new ILock(input);
-        }
-      };
-
-  public static ImmutableList<Lock> toBuildersList(Iterable<ILock> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<ILock> listFromBuilders(Iterable<Lock> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<Lock> toBuildersSet(Iterable<ILock> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<ILock> setFromBuilders(Iterable<Lock> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public Lock newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetKey() {
-    return wrapped.isSetKey();
-  }
-
-  public ILockKey getKey() {
-    return key;
-  }
-
-  public boolean isSetToken() {
-    return wrapped.isSetToken();
-  }
-
-  public String getToken() {
-    return wrapped.getToken();
-  }
-
-  public boolean isSetUser() {
-    return wrapped.isSetUser();
-  }
-
-  public String getUser() {
-    return wrapped.getUser();
-  }
-
-  public boolean isSetTimestampMs() {
-    return wrapped.isSetTimestampMs();
-  }
-
-  public long getTimestampMs() {
-    return wrapped.getTimestampMs();
-  }
-
-  public boolean isSetMessage() {
-    return wrapped.isSetMessage();
-  }
-
-  public String getMessage() {
-    return wrapped.getMessage();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof ILock)) {
-      return false;
-    }
-    ILock other = (ILock) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILockKey.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILockKey.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILockKey.java
deleted file mode 100644
index 2abc745..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ILockKey.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.LockKey;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class ILockKey {
-  private final LockKey wrapped;
-
-  private ILockKey(LockKey wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-  }
-
-  static ILockKey buildNoCopy(LockKey wrapped) {
-    return new ILockKey(wrapped);
-  }
-
-  public static ILockKey build(LockKey wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<ILockKey, LockKey> TO_BUILDER =
-      new Function<ILockKey, LockKey>() {
-        @Override
-        public LockKey apply(ILockKey input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<LockKey, ILockKey> FROM_BUILDER =
-      new Function<LockKey, ILockKey>() {
-        @Override
-        public ILockKey apply(LockKey input) {
-          return new ILockKey(input);
-        }
-      };
-
-  public static ImmutableList<LockKey> toBuildersList(Iterable<ILockKey> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<ILockKey> listFromBuilders(Iterable<LockKey> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<LockKey> toBuildersSet(Iterable<ILockKey> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<ILockKey> setFromBuilders(Iterable<LockKey> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public LockKey newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public LockKey._Fields getSetField() {
-    return wrapped.getSetField();
-  }
-
-  public IJobKey getJob() {
-    return IJobKey.build(wrapped.getJob());
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof ILockKey)) {
-      return false;
-    }
-    ILockKey other = (ILockKey) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/IPackage.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IPackage.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/IPackage.java
deleted file mode 100644
index 70b2b25..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IPackage.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.Package;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class IPackage {
-  private final Package wrapped;
-
-  private IPackage(Package wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-  }
-
-  static IPackage buildNoCopy(Package wrapped) {
-    return new IPackage(wrapped);
-  }
-
-  public static IPackage build(Package wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<IPackage, Package> TO_BUILDER =
-      new Function<IPackage, Package>() {
-        @Override
-        public Package apply(IPackage input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<Package, IPackage> FROM_BUILDER =
-      new Function<Package, IPackage>() {
-        @Override
-        public IPackage apply(Package input) {
-          return new IPackage(input);
-        }
-      };
-
-  public static ImmutableList<Package> toBuildersList(Iterable<IPackage> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<IPackage> listFromBuilders(Iterable<Package> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<Package> toBuildersSet(Iterable<IPackage> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<IPackage> setFromBuilders(Iterable<Package> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public Package newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetRole() {
-    return wrapped.isSetRole();
-  }
-
-  public String getRole() {
-    return wrapped.getRole();
-  }
-
-  public boolean isSetName() {
-    return wrapped.isSetName();
-  }
-
-  public String getName() {
-    return wrapped.getName();
-  }
-
-  public boolean isSetVersion() {
-    return wrapped.isSetVersion();
-  }
-
-  public int getVersion() {
-    return wrapped.getVersion();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof IPackage)) {
-      return false;
-    }
-    IPackage other = (IPackage) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/IQuota.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IQuota.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/IQuota.java
deleted file mode 100644
index 349cdde..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IQuota.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.Quota;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class IQuota {
-  private final Quota wrapped;
-
-  private IQuota(Quota wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-  }
-
-  static IQuota buildNoCopy(Quota wrapped) {
-    return new IQuota(wrapped);
-  }
-
-  public static IQuota build(Quota wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<IQuota, Quota> TO_BUILDER =
-      new Function<IQuota, Quota>() {
-        @Override
-        public Quota apply(IQuota input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<Quota, IQuota> FROM_BUILDER =
-      new Function<Quota, IQuota>() {
-        @Override
-        public IQuota apply(Quota input) {
-          return new IQuota(input);
-        }
-      };
-
-  public static ImmutableList<Quota> toBuildersList(Iterable<IQuota> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<IQuota> listFromBuilders(Iterable<Quota> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<Quota> toBuildersSet(Iterable<IQuota> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<IQuota> setFromBuilders(Iterable<Quota> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public Quota newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetNumCpus() {
-    return wrapped.isSetNumCpus();
-  }
-
-  public double getNumCpus() {
-    return wrapped.getNumCpus();
-  }
-
-  public boolean isSetRamMb() {
-    return wrapped.isSetRamMb();
-  }
-
-  public long getRamMb() {
-    return wrapped.getRamMb();
-  }
-
-  public boolean isSetDiskMb() {
-    return wrapped.isSetDiskMb();
-  }
-
-  public long getDiskMb() {
-    return wrapped.getDiskMb();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof IQuota)) {
-      return false;
-    }
-    IQuota other = (IQuota) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/IScheduledTask.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IScheduledTask.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/IScheduledTask.java
deleted file mode 100644
index ec58a40..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IScheduledTask.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import java.util.List;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.ScheduleStatus;
-import com.twitter.aurora.gen.ScheduledTask;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class IScheduledTask {
-  private final ScheduledTask wrapped;
-  private final IAssignedTask assignedTask;
-  private final ImmutableList<ITaskEvent> taskEvents;
-
-  private IScheduledTask(ScheduledTask wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-    this.assignedTask = !wrapped.isSetAssignedTask()
-        ? null
-        : IAssignedTask.buildNoCopy(wrapped.getAssignedTask());
-    this.taskEvents = !wrapped.isSetTaskEvents()
-        ? ImmutableList.<ITaskEvent>of()
-        : FluentIterable.from(wrapped.getTaskEvents())
-              .transform(ITaskEvent.FROM_BUILDER)
-              .toList();
-  }
-
-  static IScheduledTask buildNoCopy(ScheduledTask wrapped) {
-    return new IScheduledTask(wrapped);
-  }
-
-  public static IScheduledTask build(ScheduledTask wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<IScheduledTask, ScheduledTask> TO_BUILDER =
-      new Function<IScheduledTask, ScheduledTask>() {
-        @Override
-        public ScheduledTask apply(IScheduledTask input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<ScheduledTask, IScheduledTask> FROM_BUILDER =
-      new Function<ScheduledTask, IScheduledTask>() {
-        @Override
-        public IScheduledTask apply(ScheduledTask input) {
-          return new IScheduledTask(input);
-        }
-      };
-
-  public static ImmutableList<ScheduledTask> toBuildersList(Iterable<IScheduledTask> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<IScheduledTask> listFromBuilders(Iterable<ScheduledTask> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<ScheduledTask> toBuildersSet(Iterable<IScheduledTask> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<IScheduledTask> setFromBuilders(Iterable<ScheduledTask> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public ScheduledTask newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetAssignedTask() {
-    return wrapped.isSetAssignedTask();
-  }
-
-  public IAssignedTask getAssignedTask() {
-    return assignedTask;
-  }
-
-  public ScheduleStatus getStatus() {
-    return wrapped.getStatus();
-  }
-
-  public boolean isSetFailureCount() {
-    return wrapped.isSetFailureCount();
-  }
-
-  public int getFailureCount() {
-    return wrapped.getFailureCount();
-  }
-
-  public boolean isSetTaskEvents() {
-    return wrapped.isSetTaskEvents();
-  }
-
-  public List<ITaskEvent> getTaskEvents() {
-    return taskEvents;
-  }
-
-  public boolean isSetAncestorId() {
-    return wrapped.isSetAncestorId();
-  }
-
-  public String getAncestorId() {
-    return wrapped.getAncestorId();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof IScheduledTask)) {
-      return false;
-    }
-    IScheduledTask other = (IScheduledTask) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskConfig.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskConfig.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskConfig.java
deleted file mode 100644
index 7fa223d..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskConfig.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import java.util.Map;
-import java.util.Set;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.TaskConfig;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class ITaskConfig {
-  private final TaskConfig wrapped;
-  private final IIdentity owner;
-  private final ImmutableSet<IConstraint> constraints;
-  private final ImmutableSet<String> requestedPorts;
-  private final ImmutableMap<String, String> taskLinks;
-  private final ImmutableSet<IPackage> packages;
-  private final IExecutorConfig executorConfig;
-
-  private ITaskConfig(TaskConfig wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-    this.owner = !wrapped.isSetOwner()
-        ? null
-        : IIdentity.buildNoCopy(wrapped.getOwner());
-    this.constraints = !wrapped.isSetConstraints()
-        ? ImmutableSet.<IConstraint>of()
-        : FluentIterable.from(wrapped.getConstraints())
-              .transform(IConstraint.FROM_BUILDER)
-              .toSet();
-    this.requestedPorts = !wrapped.isSetRequestedPorts()
-        ? ImmutableSet.<String>of()
-        : ImmutableSet.copyOf(wrapped.getRequestedPorts());
-    this.taskLinks = !wrapped.isSetTaskLinks()
-        ? ImmutableMap.<String, String>of()
-        : ImmutableMap.copyOf(wrapped.getTaskLinks());
-    this.packages = !wrapped.isSetPackages()
-        ? ImmutableSet.<IPackage>of()
-        : FluentIterable.from(wrapped.getPackages())
-              .transform(IPackage.FROM_BUILDER)
-              .toSet();
-    this.executorConfig = !wrapped.isSetExecutorConfig()
-        ? null
-        : IExecutorConfig.buildNoCopy(wrapped.getExecutorConfig());
-  }
-
-  static ITaskConfig buildNoCopy(TaskConfig wrapped) {
-    return new ITaskConfig(wrapped);
-  }
-
-  public static ITaskConfig build(TaskConfig wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<ITaskConfig, TaskConfig> TO_BUILDER =
-      new Function<ITaskConfig, TaskConfig>() {
-        @Override
-        public TaskConfig apply(ITaskConfig input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<TaskConfig, ITaskConfig> FROM_BUILDER =
-      new Function<TaskConfig, ITaskConfig>() {
-        @Override
-        public ITaskConfig apply(TaskConfig input) {
-          return new ITaskConfig(input);
-        }
-      };
-
-  public static ImmutableList<TaskConfig> toBuildersList(Iterable<ITaskConfig> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<ITaskConfig> listFromBuilders(Iterable<TaskConfig> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<TaskConfig> toBuildersSet(Iterable<ITaskConfig> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<ITaskConfig> setFromBuilders(Iterable<TaskConfig> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public TaskConfig newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetOwner() {
-    return wrapped.isSetOwner();
-  }
-
-  public IIdentity getOwner() {
-    return owner;
-  }
-
-  public boolean isSetEnvironment() {
-    return wrapped.isSetEnvironment();
-  }
-
-  public String getEnvironment() {
-    return wrapped.getEnvironment();
-  }
-
-  public boolean isSetJobName() {
-    return wrapped.isSetJobName();
-  }
-
-  public String getJobName() {
-    return wrapped.getJobName();
-  }
-
-  public boolean isSetIsService() {
-    return wrapped.isSetIsService();
-  }
-
-  public boolean isIsService() {
-    return wrapped.isIsService();
-  }
-
-  public boolean isSetNumCpus() {
-    return wrapped.isSetNumCpus();
-  }
-
-  public double getNumCpus() {
-    return wrapped.getNumCpus();
-  }
-
-  public boolean isSetRamMb() {
-    return wrapped.isSetRamMb();
-  }
-
-  public long getRamMb() {
-    return wrapped.getRamMb();
-  }
-
-  public boolean isSetDiskMb() {
-    return wrapped.isSetDiskMb();
-  }
-
-  public long getDiskMb() {
-    return wrapped.getDiskMb();
-  }
-
-  public boolean isSetPriority() {
-    return wrapped.isSetPriority();
-  }
-
-  public int getPriority() {
-    return wrapped.getPriority();
-  }
-
-  public boolean isSetMaxTaskFailures() {
-    return wrapped.isSetMaxTaskFailures();
-  }
-
-  public int getMaxTaskFailures() {
-    return wrapped.getMaxTaskFailures();
-  }
-
-  public boolean isSetInstanceIdDEPRECATED() {
-    return wrapped.isSetInstanceIdDEPRECATED();
-  }
-
-  public int getInstanceIdDEPRECATED() {
-    return wrapped.getInstanceIdDEPRECATED();
-  }
-
-  public boolean isSetProduction() {
-    return wrapped.isSetProduction();
-  }
-
-  public boolean isProduction() {
-    return wrapped.isProduction();
-  }
-
-  public boolean isSetConstraints() {
-    return wrapped.isSetConstraints();
-  }
-
-  public Set<IConstraint> getConstraints() {
-    return constraints;
-  }
-
-  public boolean isSetRequestedPorts() {
-    return wrapped.isSetRequestedPorts();
-  }
-
-  public Set<String> getRequestedPorts() {
-    return requestedPorts;
-  }
-
-  public boolean isSetTaskLinks() {
-    return wrapped.isSetTaskLinks();
-  }
-
-  public Map<String, String> getTaskLinks() {
-    return taskLinks;
-  }
-
-  public boolean isSetContactEmail() {
-    return wrapped.isSetContactEmail();
-  }
-
-  public String getContactEmail() {
-    return wrapped.getContactEmail();
-  }
-
-  public boolean isSetPackages() {
-    return wrapped.isSetPackages();
-  }
-
-  public Set<IPackage> getPackages() {
-    return packages;
-  }
-
-  public boolean isSetExecutorConfig() {
-    return wrapped.isSetExecutorConfig();
-  }
-
-  public IExecutorConfig getExecutorConfig() {
-    return executorConfig;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof ITaskConfig)) {
-      return false;
-    }
-    ITaskConfig other = (ITaskConfig) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskConstraint.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskConstraint.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskConstraint.java
deleted file mode 100644
index 9807008..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskConstraint.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.TaskConstraint;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class ITaskConstraint {
-  private final TaskConstraint wrapped;
-
-  private ITaskConstraint(TaskConstraint wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-  }
-
-  static ITaskConstraint buildNoCopy(TaskConstraint wrapped) {
-    return new ITaskConstraint(wrapped);
-  }
-
-  public static ITaskConstraint build(TaskConstraint wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<ITaskConstraint, TaskConstraint> TO_BUILDER =
-      new Function<ITaskConstraint, TaskConstraint>() {
-        @Override
-        public TaskConstraint apply(ITaskConstraint input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<TaskConstraint, ITaskConstraint> FROM_BUILDER =
-      new Function<TaskConstraint, ITaskConstraint>() {
-        @Override
-        public ITaskConstraint apply(TaskConstraint input) {
-          return new ITaskConstraint(input);
-        }
-      };
-
-  public static ImmutableList<TaskConstraint> toBuildersList(Iterable<ITaskConstraint> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<ITaskConstraint> listFromBuilders(Iterable<TaskConstraint> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<TaskConstraint> toBuildersSet(Iterable<ITaskConstraint> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<ITaskConstraint> setFromBuilders(Iterable<TaskConstraint> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public TaskConstraint newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public TaskConstraint._Fields getSetField() {
-    return wrapped.getSetField();
-  }
-
-  public IValueConstraint getValue() {
-    return IValueConstraint.build(wrapped.getValue());
-  }
-
-  public ILimitConstraint getLimit() {
-    return ILimitConstraint.build(wrapped.getLimit());
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof ITaskConstraint)) {
-      return false;
-    }
-    ITaskConstraint other = (ITaskConstraint) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskEvent.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskEvent.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskEvent.java
deleted file mode 100644
index 5e1de07..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/ITaskEvent.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.ScheduleStatus;
-import com.twitter.aurora.gen.TaskEvent;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class ITaskEvent {
-  private final TaskEvent wrapped;
-
-  private ITaskEvent(TaskEvent wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-  }
-
-  static ITaskEvent buildNoCopy(TaskEvent wrapped) {
-    return new ITaskEvent(wrapped);
-  }
-
-  public static ITaskEvent build(TaskEvent wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<ITaskEvent, TaskEvent> TO_BUILDER =
-      new Function<ITaskEvent, TaskEvent>() {
-        @Override
-        public TaskEvent apply(ITaskEvent input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<TaskEvent, ITaskEvent> FROM_BUILDER =
-      new Function<TaskEvent, ITaskEvent>() {
-        @Override
-        public ITaskEvent apply(TaskEvent input) {
-          return new ITaskEvent(input);
-        }
-      };
-
-  public static ImmutableList<TaskEvent> toBuildersList(Iterable<ITaskEvent> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<ITaskEvent> listFromBuilders(Iterable<TaskEvent> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<TaskEvent> toBuildersSet(Iterable<ITaskEvent> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<ITaskEvent> setFromBuilders(Iterable<TaskEvent> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public TaskEvent newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetTimestamp() {
-    return wrapped.isSetTimestamp();
-  }
-
-  public long getTimestamp() {
-    return wrapped.getTimestamp();
-  }
-
-  public ScheduleStatus getStatus() {
-    return wrapped.getStatus();
-  }
-
-  public boolean isSetMessage() {
-    return wrapped.isSetMessage();
-  }
-
-  public String getMessage() {
-    return wrapped.getMessage();
-  }
-
-  public boolean isSetScheduler() {
-    return wrapped.isSetScheduler();
-  }
-
-  public String getScheduler() {
-    return wrapped.getScheduler();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof ITaskEvent)) {
-      return false;
-    }
-    ITaskEvent other = (ITaskEvent) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/java/com/twitter/aurora/scheduler/storage/entities/IValueConstraint.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IValueConstraint.java b/src/main/java/com/twitter/aurora/scheduler/storage/entities/IValueConstraint.java
deleted file mode 100644
index d380770..0000000
--- a/src/main/java/com/twitter/aurora/scheduler/storage/entities/IValueConstraint.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Copyright 2013 Twitter, 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 com.twitter.aurora.scheduler.storage.entities;
-
-import java.util.Set;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-import com.twitter.aurora.gen.ValueConstraint;
-
-/**
- * An immutable wrapper class.
- * <p>
- * This code is auto-generated, and should not be directly modified.
- * <p>
- * Yes, you're right, it shouldn't be checked in.  We'll get there, I promise.
- */
-public final class IValueConstraint {
-  private final ValueConstraint wrapped;
-  private final ImmutableSet<String> values;
-
-  private IValueConstraint(ValueConstraint wrapped) {
-    this.wrapped = Preconditions.checkNotNull(wrapped);
-    this.values = !wrapped.isSetValues()
-        ? ImmutableSet.<String>of()
-        : ImmutableSet.copyOf(wrapped.getValues());
-  }
-
-  static IValueConstraint buildNoCopy(ValueConstraint wrapped) {
-    return new IValueConstraint(wrapped);
-  }
-
-  public static IValueConstraint build(ValueConstraint wrapped) {
-    return buildNoCopy(wrapped.deepCopy());
-  }
-
-  public static final Function<IValueConstraint, ValueConstraint> TO_BUILDER =
-      new Function<IValueConstraint, ValueConstraint>() {
-        @Override
-        public ValueConstraint apply(IValueConstraint input) {
-          return input.newBuilder();
-        }
-      };
-
-  public static final Function<ValueConstraint, IValueConstraint> FROM_BUILDER =
-      new Function<ValueConstraint, IValueConstraint>() {
-        @Override
-        public IValueConstraint apply(ValueConstraint input) {
-          return new IValueConstraint(input);
-        }
-      };
-
-  public static ImmutableList<ValueConstraint> toBuildersList(Iterable<IValueConstraint> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toList();
-  }
-
-  public static ImmutableList<IValueConstraint> listFromBuilders(Iterable<ValueConstraint> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toList();
-  }
-
-  public static ImmutableSet<ValueConstraint> toBuildersSet(Iterable<IValueConstraint> w) {
-    return FluentIterable.from(w).transform(TO_BUILDER).toSet();
-  }
-
-  public static ImmutableSet<IValueConstraint> setFromBuilders(Iterable<ValueConstraint> b) {
-    return FluentIterable.from(b).transform(FROM_BUILDER).toSet();
-  }
-
-  public ValueConstraint newBuilder() {
-    return wrapped.deepCopy();
-  }
-
-  public boolean isSetNegated() {
-    return wrapped.isSetNegated();
-  }
-
-  public boolean isNegated() {
-    return wrapped.isNegated();
-  }
-
-  public boolean isSetValues() {
-    return wrapped.isSetValues();
-  }
-
-  public Set<String> getValues() {
-    return values;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof IValueConstraint)) {
-      return false;
-    }
-    IValueConstraint other = (IValueConstraint) o;
-    return wrapped.equals(other.wrapped);
-  }
-
-  @Override
-  public int hashCode() {
-    return wrapped.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return wrapped.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/87047365/src/main/python/twitter/aurora/tools/java/organize_imports.py
----------------------------------------------------------------------
diff --git a/src/main/python/twitter/aurora/tools/java/organize_imports.py b/src/main/python/twitter/aurora/tools/java/organize_imports.py
new file mode 100644
index 0000000..48a0821
--- /dev/null
+++ b/src/main/python/twitter/aurora/tools/java/organize_imports.py
@@ -0,0 +1,130 @@
+#!/bin/env python
+'''
+Organizes a java source file's import statements in a way that pleases Twitter's checkstyle
+configuration.
+This expects exactly one argument: the name of the file to modify with preferred import
+ordering.
+'''
+
+from __future__ import print_function
+
+import re
+import sys
+from collections import defaultdict
+
+
+IMPORT_RE = re.compile('import(?: static)? (.*);')
+def get_group(import_statement):
+  matcher = IMPORT_RE.match(import_statement)
+  assert matcher, 'Could not parse import statement: %s' % import_statement
+  class_name = matcher.group(1)
+  if class_name.startswith('com.twitter'):
+    group = 'com.twitter'
+  else:
+    group = class_name.split('.')[0]
+  return group
+
+
+def index_by_group(import_statements):
+  groups = defaultdict(list)
+  for line in import_statements:
+    groups[get_group(line)].append(line)
+  return groups
+
+
+IMPORT_CLASS_RE = re.compile(
+    'import(?: static)? (?P<outer>[^A-Z]*[A-Z]\w+)(?:\.(?P<inners>[\w][^;]*))?')
+def get_all_group_lines(import_groups):
+  if not import_groups:
+    return []
+
+  def get_group_lines(group):
+    def comparator(x, y):
+      # These shenanigans are used to properly order imports for inner classes.
+      # So we get ordering like:
+      # import com.foo.Bar;
+      # import com.foo.Bar.Baz;
+      # (this is not lexicographical, so normal sort won't suffice)
+      x_m = IMPORT_CLASS_RE.match(x)
+      y_m = IMPORT_CLASS_RE.match(y)
+      if x_m.group('outer') == y_m.group('outer'):
+        return cmp(x_m.group('inners'), y_m.group('inners'))
+      else:
+        return cmp(x, y)
+    lines = sorted(import_groups[group], comparator)
+    lines.append('')
+    return lines 
+
+  all_lines = []
+  explicit_groups = ['java', 'javax', 'scala', 'com', 'net', 'org', 'com.twitter']
+  for group in explicit_groups:
+    if group in import_groups:
+      all_lines += get_group_lines(group)
+
+  # Gather remaining groups.
+  remaining_groups = sorted(set(import_groups.keys()) - set(explicit_groups))
+  for group in remaining_groups:
+    all_lines += get_group_lines(group)
+  return all_lines
+
+
+if len(sys.argv) != 2:
+  print('usage: %s FILE' % sys.argv[0])
+  sys.exit(1)
+
+BEFORE_IMPORTS = 'before_imports'
+IMPORTS = 'imports'
+STATIC_IMPORTS = 'static_imports'
+AFTER_IMPORTS = 'after_imports'
+
+print('Organizing imports in %s' % sys.argv[1])
+lines_before_imports = []
+import_lines = []
+static_import_lines = []
+lines_after_imports = []
+with open(sys.argv[1], 'r') as f:
+  position = BEFORE_IMPORTS
+  for line in f:
+    line = line.rstrip()
+    if position == BEFORE_IMPORTS:
+      if line.startswith('import'):
+        position = IMPORTS
+      else:
+        lines_before_imports.append(line)
+    if position == IMPORTS:
+      if line.startswith('import static'):
+        position = STATIC_IMPORTS
+      elif line.startswith('import'):
+        import_lines.append(line)
+      elif line.strip():
+        position = AFTER_IMPORTS
+    if position == STATIC_IMPORTS:
+      if line.startswith('import static'):
+        static_import_lines.append(line)
+      elif line.strip():
+        position = AFTER_IMPORTS
+    if position == AFTER_IMPORTS:
+      lines_after_imports.append(line)
+
+import_groups = index_by_group(import_lines)
+static_import_groups = index_by_group(static_import_lines)
+
+def ensure_line_padding(lines):
+  if lines and lines[-1] != '':
+    lines.append('')
+  return lines
+
+file_lines = lines_before_imports
+if import_groups:
+  ensure_line_padding(file_lines)
+  file_lines += get_all_group_lines(import_groups)
+if static_import_groups:
+  ensure_line_padding(file_lines)
+  file_lines += get_all_group_lines(static_import_groups)
+if lines_after_imports:
+  ensure_line_padding(file_lines)
+  file_lines += lines_after_imports
+
+with open(sys.argv[1], 'w') as f:
+  for line in file_lines:
+    print(line, file=f)


[4/8] git commit: Do not compile Haskell support.

Posted by ke...@apache.org.
Do not compile Haskell support.

Our bootstrapped thrift compiler does not need Haskell support. This change
removes some of the complexity needed to bootstrap the thrift compiler.


Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/2f79ec35
Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/2f79ec35
Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/2f79ec35

Branch: refs/heads/master
Commit: 2f79ec354e549490d644c96f2e8ef53e172e4452
Parents: 29d39b1
Author: Zameer Manji <zm...@twitter.com>
Authored: Tue Nov 26 15:01:40 2013 -0800
Committer: Zameer Manji <zm...@twitter.com>
Committed: Tue Nov 26 15:01:40 2013 -0800

----------------------------------------------------------------------
 build-support/thrift/Makefile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/2f79ec35/build-support/thrift/Makefile
----------------------------------------------------------------------
diff --git a/build-support/thrift/Makefile b/build-support/thrift/Makefile
index 459cd36..10f201f 100644
--- a/build-support/thrift/Makefile
+++ b/build-support/thrift/Makefile
@@ -21,7 +21,7 @@ all: $(THRIFT)
 
 $(THRIFT):
 	curl $(THRIFT_URL) | tar zxv
-	(cd $(THRIFT_DISTNAME) && ./configure --without-php --without-php_extension --without-ruby)
+	(cd $(THRIFT_DISTNAME) && ./configure --without-php --without-php_extension --without-ruby --without-haskell)
 	(cd $(THRIFT_DISTNAME) && make -j4)
 
 clean:


[8/8] git commit: Remove test for require_contact_email

Posted by ke...@apache.org.
Remove test for require_contact_email


Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/c72c92cf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/c72c92cf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/c72c92cf

Branch: refs/heads/master
Commit: c72c92cf3edcf1098862bfb449c00dac1f364474
Parents: 130d466
Author: Kevin Sweeney <ks...@twitter.com>
Authored: Tue Dec 3 12:01:41 2013 -0800
Committer: Kevin Sweeney <ks...@twitter.com>
Committed: Tue Dec 3 12:02:19 2013 -0800

----------------------------------------------------------------------
 .../configuration/ConfigurationManagerTest.java | 20 --------------------
 1 file changed, 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c72c92cf/src/test/java/com/twitter/aurora/scheduler/configuration/ConfigurationManagerTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/com/twitter/aurora/scheduler/configuration/ConfigurationManagerTest.java b/src/test/java/com/twitter/aurora/scheduler/configuration/ConfigurationManagerTest.java
index 97b93f1..688d106 100644
--- a/src/test/java/com/twitter/aurora/scheduler/configuration/ConfigurationManagerTest.java
+++ b/src/test/java/com/twitter/aurora/scheduler/configuration/ConfigurationManagerTest.java
@@ -103,24 +103,4 @@ public class ConfigurationManagerTest {
     assertTrue(copy.isSetKey());
     assertEquals(DEFAULT_ENVIRONMENT, copy.getKey().getEnvironment());
   }
-
-  @Test
-  public void testRequiresContactEmail() throws Exception {
-    JobConfiguration copy = UNSANITIZED_JOB_CONFIGURATION.deepCopy();
-    copy.getTaskConfig().unsetContactEmail();
-    expectRejected(copy);
-    copy.getTaskConfig().setContactEmail("invalid");
-    expectRejected(copy);
-    copy.getTaskConfig().setContactEmail("jim@aol.com");
-    expectRejected(copy);
-  }
-
-  private void expectRejected(JobConfiguration job) {
-    try {
-      ConfigurationManager.validateAndPopulate(IJobConfiguration.build(job));
-      fail();
-    } catch (ConfigurationManager.TaskDescriptionException e) {
-      // expected
-    }
-  }
 }


[6/8] git commit: Remove socksipy dependency from http_signaler.

Posted by ke...@apache.org.
Remove socksipy dependency from http_signaler.

Remove socksipy dependency from http_signaler.
Fixes small issue in environment bootstrapping script.
Increments twitter.aurora artifact version numbers.


Project: http://git-wip-us.apache.org/repos/asf/incubator-aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-aurora/commit/52b566cb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-aurora/tree/52b566cb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-aurora/diff/52b566cb

Branch: refs/heads/master
Commit: 52b566cb6e49289dfc978969b50faad70e245440
Parents: ce669c1
Author: Brian Wickman <wi...@twitter.com>
Authored: Mon Dec 2 10:41:19 2013 -0800
Committer: Brian Wickman <wi...@twitter.com>
Committed: Mon Dec 2 10:41:19 2013 -0800

----------------------------------------------------------------------
 .gitignore                                        |  1 +
 examples/vagrant/provision-dev-environment.sh     |  2 +-
 src/main/python/twitter/aurora/BUILD.thirdparty   |  1 -
 src/main/python/twitter/aurora/client/BUILD       |  2 +-
 src/main/python/twitter/aurora/common/BUILD       |  6 +-----
 .../python/twitter/aurora/common/http_signaler.py | 18 ++++++++++--------
 src/main/python/twitter/aurora/config/BUILD       |  2 +-
 src/main/python/twitter/aurora/executor/BUILD     |  2 +-
 8 files changed, 16 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/52b566cb/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index f680e29..3731eeb 100644
--- a/.gitignore
+++ b/.gitignore
@@ -3,6 +3,7 @@
 *.iws
 *.pyc
 *~
+.coverage
 .gradle/
 .pants.*
 .reviewboardrc

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/52b566cb/examples/vagrant/provision-dev-environment.sh
----------------------------------------------------------------------
diff --git a/examples/vagrant/provision-dev-environment.sh b/examples/vagrant/provision-dev-environment.sh
index f2e53ab..957f93d 100755
--- a/examples/vagrant/provision-dev-environment.sh
+++ b/examples/vagrant/provision-dev-environment.sh
@@ -68,4 +68,4 @@ EOF
   done
 popd
 
-sudo chown -R vagrant.vagrant mesos mesos-build aurora .pex
+sudo chown -R vagrant.vagrant mesos mesos-build aurora

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/52b566cb/src/main/python/twitter/aurora/BUILD.thirdparty
----------------------------------------------------------------------
diff --git a/src/main/python/twitter/aurora/BUILD.thirdparty b/src/main/python/twitter/aurora/BUILD.thirdparty
index bcfa2af..87156ea 100644
--- a/src/main/python/twitter/aurora/BUILD.thirdparty
+++ b/src/main/python/twitter/aurora/BUILD.thirdparty
@@ -15,5 +15,4 @@ make_dep('mox', '0.5.3')
 make_dep('psutil', '1.1.2')
 make_dep('pystachio', '0.7.2')
 make_dep('pyyaml', '3.10', 'PyYAML')
-make_dep('socksipy-branch', '1.01', 'SocksiPy-Branch')
 make_dep('thrift', '0.9.0')

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/52b566cb/src/main/python/twitter/aurora/client/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/twitter/aurora/client/BUILD b/src/main/python/twitter/aurora/client/BUILD
index ded3cdc..a96a801 100644
--- a/src/main/python/twitter/aurora/client/BUILD
+++ b/src/main/python/twitter/aurora/client/BUILD
@@ -68,7 +68,7 @@ python_library(
   ],
   provides = setup_py(
     name = 'twitter.aurora.client',
-    version = '0.1.1',
+    version = '0.1.2',
   ).with_binaries(
     aurora_admin = pants('src/main/python/twitter/aurora/client/bin:aurora_admin'),
     aurora_client = pants('src/main/python/twitter/aurora/client/bin:aurora_client'),

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/52b566cb/src/main/python/twitter/aurora/common/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/twitter/aurora/common/BUILD b/src/main/python/twitter/aurora/common/BUILD
index aeac6fd..c948f94 100644
--- a/src/main/python/twitter/aurora/common/BUILD
+++ b/src/main/python/twitter/aurora/common/BUILD
@@ -37,10 +37,6 @@ python_library(
   name = 'http_signaler',
   sources = ['http_signaler.py'],
   dependencies = [
-    # TODO(wickman) This dependency needs to be removed.  It is a scope leak
-    # for monkeypatched SOCKS usage and should be moved into an org-specific
-    # HttpSignaler and injected at the AuroraClientAPI level.
-    pants('src/main/python/twitter/aurora/BUILD.thirdparty:socksipy-branch'),
     pants('aurora/twitterdeps/src/python/twitter/common/lang'),
     pants('aurora/twitterdeps/src/python/twitter/common/log'),
   ]
@@ -58,7 +54,7 @@ python_library(
   ],
   provides = setup_py(
     name = 'twitter.aurora.common',
-    version = '0.1.1',
+    version = '0.1.2',
     description = 'Aurora common libraries.',
     license = 'Apache License, Version 2.0',
   )

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/52b566cb/src/main/python/twitter/aurora/common/http_signaler.py
----------------------------------------------------------------------
diff --git a/src/main/python/twitter/aurora/common/http_signaler.py b/src/main/python/twitter/aurora/common/http_signaler.py
index 47c86d5..778ddc4 100644
--- a/src/main/python/twitter/aurora/common/http_signaler.py
+++ b/src/main/python/twitter/aurora/common/http_signaler.py
@@ -14,10 +14,6 @@ else:
   import urllib2 as urllib_request
   from urllib2 import URLError, HTTPError
 
-# TODO(wickman) This is an abstraction leak -- this should be fixed upstream by
-# MESOS-3710
-import socks
-
 
 class HttpSignaler(object):
   """Simple HTTP endpoint wrapper to check health or trigger quitquitquit/abortabortabort"""
@@ -43,16 +39,22 @@ class HttpSignaler(object):
     """Request an HTTP endpoint with a GET request (or POST if data is not None)"""
     url = self.url(endpoint)
     log.debug("%s: %s %s" % (self.__class__.__name__, 'GET' if data is None else 'POST', url))
+
+    def raise_error(reason):
+      raise self.QueryError('Failed to signal %s: %s' % (self.url(endpoint), reason))
+
     try:
       with contextlib.closing(
           self.opener(url, data, timeout=self._timeout_secs)) as fp:
         return fp.read()
-    except (URLError, HTTPError, HTTPException, SocketTimeout, socks.GeneralProxyError) as e:
+    except (HTTPException, SocketTimeout) as e:
       # the type of an HTTPException is typically more useful than its contents (since for example
       # BadStatusLines are often empty). likewise with socket.timeout.
-      err = e.__class__.__name__ if isinstance(e, (HTTPException, SocketTimeout)) else e
-      reason = 'Failed to signal %s: %s' % (self.url(endpoint), err)
-      raise self.QueryError(reason)
+      raise_error('Error within %s' % e.__class__.__name__)
+    except (URLError, HTTPError) as e:
+      raise_error(e)
+    except Exception as e:
+      raise_error('Unexpected error: %s' % e)
 
   def __call__(self, endpoint, use_post_method=False, expected_response=None):
     """Returns a (boolean, string|None) tuple of (call success, failure reason)"""

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/52b566cb/src/main/python/twitter/aurora/config/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/twitter/aurora/config/BUILD b/src/main/python/twitter/aurora/config/BUILD
index 9cdf112..0462158 100644
--- a/src/main/python/twitter/aurora/config/BUILD
+++ b/src/main/python/twitter/aurora/config/BUILD
@@ -34,7 +34,7 @@ python_library(
   ],
   provides = setup_py(
     name = 'twitter.aurora.config',
-    version = '0.1.1',
+    version = '0.1.2',
     description = 'Aurora/Thermos Pystachio schemas for describing job configurations.',
     license = 'Apache License, Version 2.0',
   )

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/52b566cb/src/main/python/twitter/aurora/executor/BUILD
----------------------------------------------------------------------
diff --git a/src/main/python/twitter/aurora/executor/BUILD b/src/main/python/twitter/aurora/executor/BUILD
index 47614da..02d34f0 100644
--- a/src/main/python/twitter/aurora/executor/BUILD
+++ b/src/main/python/twitter/aurora/executor/BUILD
@@ -128,7 +128,7 @@ python_library(
   ],
   provides = setup_py(
     name = 'twitter.aurora.executor',
-    version = '0.1.2',
+    version = '0.1.3',
   ).with_binaries(
     gc_executor = pants('src/main/python/twitter/aurora/executor/bin:gc_executor'),
     thermos_executor = pants('src/main/python/twitter/aurora/executor/bin:thermos_executor'),