You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2012/01/11 23:33:29 UTC

svn commit: r1230293 - in /incubator/mesos/trunk/src: common/attributes.cpp common/attributes.hpp common/cache.hpp common/timer.hpp common/values.cpp common/values.hpp tests/attributes_test.cpp tests/values_tests.cpp

Author: benh
Date: Wed Jan 11 22:33:29 2012
New Revision: 1230293

URL: http://svn.apache.org/viewvc?rev=1230293&view=rev
Log:
Failed to add some new files, my apologies.

Added:
    incubator/mesos/trunk/src/common/attributes.cpp
    incubator/mesos/trunk/src/common/attributes.hpp
    incubator/mesos/trunk/src/common/cache.hpp
    incubator/mesos/trunk/src/common/timer.hpp
    incubator/mesos/trunk/src/common/values.cpp
    incubator/mesos/trunk/src/common/values.hpp
    incubator/mesos/trunk/src/tests/attributes_test.cpp
    incubator/mesos/trunk/src/tests/values_tests.cpp

Added: incubator/mesos/trunk/src/common/attributes.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/attributes.cpp?rev=1230293&view=auto
==============================================================================
--- incubator/mesos/trunk/src/common/attributes.cpp (added)
+++ incubator/mesos/trunk/src/common/attributes.cpp Wed Jan 11 22:33:29 2012
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.
+ */
+
+#include <iostream>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "common/attributes.hpp"
+#include "common/foreach.hpp"
+#include "common/strings.hpp"
+
+using std::ostream;
+using std::string;
+using std::vector;
+
+
+namespace mesos {
+namespace internal {
+
+Attribute Attributes::parse(const std::string& name, const std::string& text)
+{
+  Attribute attribute;
+  Try<Value> result = values::parse(text);
+
+  if (result.isError()) {
+    LOG(FATAL) << "Failed to parse attribute " << name
+               << " text " << text
+               << " error " << result.error();
+  } else{
+    Value value = result.get();
+    attribute.set_name(name);
+
+    if (value.type() == Value::RANGES) {
+      attribute.set_type(Value::RANGES);
+      attribute.mutable_ranges()->MergeFrom(value.ranges());
+    } else if (value.type() == Value::TEXT) {
+      attribute.set_type(Value::TEXT);
+      attribute.mutable_text()->MergeFrom(value.text());
+    } else if (value.type() == Value::SCALAR) {
+      attribute.set_type(Value::SCALAR);
+      attribute.mutable_scalar()->MergeFrom(value.scalar());
+    } else {
+      LOG(FATAL) << "Bad type for attribute " << name
+                 << " text " << text
+                 << " type " << value.type();
+    }
+  }
+
+  return attribute;
+}
+
+
+Attributes Attributes::parse(const string& s)
+{
+  // Tokenize and parse the value of "attributes".
+  Attributes attributes;
+
+  vector<string> tokens = strings::split(s, ";\n");
+
+  for (int i = 0; i < tokens.size(); i++) {
+    const vector<string>& pairs = strings::split(tokens[i], ":");
+    if (pairs.size() != 2) {
+      LOG(FATAL) << "Bad value for attributes, missing ':' within " << pairs[0];
+    }
+
+    attributes.add(parse(pairs[0], pairs[1]));
+  }
+
+  return attributes;
+}
+
+} // namespace internal {
+} // namespace mesos {

Added: incubator/mesos/trunk/src/common/attributes.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/attributes.hpp?rev=1230293&view=auto
==============================================================================
--- incubator/mesos/trunk/src/common/attributes.hpp (added)
+++ incubator/mesos/trunk/src/common/attributes.hpp Wed Jan 11 22:33:29 2012
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.
+ */
+
+#ifndef __ATTRIBUTES_HPP__
+#define __ATTRIBUTES_HPP__
+
+#include <iterator>
+#include <string>
+
+#include <mesos/mesos.hpp>
+
+#include "common/foreach.hpp"
+#include "common/option.hpp"
+#include "common/values.hpp"
+
+namespace mesos {
+namespace internal {
+
+
+class Attributes
+{
+public:
+  Attributes() {}
+
+  Attributes(const google::protobuf::RepeatedPtrField<Attribute>& _attributes)
+  {
+    attributes.MergeFrom(_attributes);
+  }
+
+  Attributes(const Attributes& that)
+  {
+    attributes.MergeFrom(that.attributes);
+  }
+
+  Attributes& operator = (const Attributes& that)
+  {
+    if (this != &that) {
+      attributes.Clear();
+      attributes.MergeFrom(that.attributes);
+    }
+
+    return *this;
+  }
+
+  size_t size() const
+  {
+    return attributes.size();
+  }
+
+  // Using this operator makes it easy to copy a attributes object into
+  // a protocol buffer field.
+  operator const google::protobuf::RepeatedPtrField<Attribute>& () const
+  {
+    return attributes;
+  }
+
+  void add(const Attribute& attribute)
+  {
+    attributes.Add()->MergeFrom(attribute);
+  }
+
+  const Attribute get(int index) const
+  {
+    return attributes.Get(index);
+  }
+
+  template <typename T>
+  T get(const std::string& name, const T& t) const;
+
+  typedef google::protobuf::RepeatedPtrField<Attribute>::iterator
+  iterator;
+
+  typedef google::protobuf::RepeatedPtrField<Attribute>::const_iterator
+  const_iterator;
+
+  iterator begin() { return attributes.begin(); }
+  iterator end() { return attributes.end(); }
+
+  const_iterator begin() const { return attributes.begin(); }
+  const_iterator end() const { return attributes.end(); }
+
+  static Attribute parse(const std::string& name, const std::string& value);
+  static Attributes parse(const std::string& s);
+
+  static bool isValid(const Attribute& attribute)
+  {
+    if (!attribute.has_name() ||
+        attribute.name() == "" ||
+        !attribute.has_type() ||
+        !Value::Type_IsValid(attribute.type())) {
+      return false;
+    }
+
+    if (attribute.type() == Value::SCALAR) {
+      return attribute.has_scalar();
+    } else if (attribute.type() == Value::RANGES) {
+      return attribute.has_ranges();
+    } else if (attribute.type() == Value::TEXT) {
+      return attribute.has_text();
+    } else if (attribute.type() == Value::SET) {
+      // Attributes doesn't support set.
+      return false;
+    }
+
+    return false;
+  }
+
+private:
+  google::protobuf::RepeatedPtrField<Attribute> attributes;
+};
+
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __ATTRIBUTES_HPP__

Added: incubator/mesos/trunk/src/common/cache.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/cache.hpp?rev=1230293&view=auto
==============================================================================
--- incubator/mesos/trunk/src/common/cache.hpp (added)
+++ incubator/mesos/trunk/src/common/cache.hpp Wed Jan 11 22:33:29 2012
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.
+ */
+
+#ifndef __CACHE_HPP__
+#define __CACHE_HPP__
+
+#include <functional>
+#include <iostream>
+#include <list>
+#include <map>
+
+#include <tr1/functional>
+
+#include "common/option.hpp"
+
+// Forward declaration.
+template <typename Key, typename Value>
+class cache;
+
+// Outputs the key/value pairs from least to most-recently used.
+template <typename Key, typename Value>
+std::ostream& operator << (
+    std::ostream& stream,
+    const cache<Key, Value>& c);
+
+
+// Provides a least-recently used (LRU) cache of some predefined
+// capacity. A "write" and a "read" both count as uses.
+template <typename Key, typename Value>
+class cache
+{
+public:
+  typedef std::list<Key> list;
+  typedef std::tr1::unordered_map<
+    Key, std::pair<Value, typename list::iterator> > map;
+
+  explicit cache(int _capacity) : capacity(_capacity) {}
+
+  void put(const Key& key, const Value& value)
+  {
+    typename map::iterator i = values.find(key);
+    if (i == values.end()) {
+      insert(key, value);
+    } else {
+      (*i).second.first = value;
+      use(i);
+    }
+  }
+
+  Option<Value> get(const Key& key)
+  {
+    typename map::iterator i = values.find(key);
+
+    if (i != values.end()) {
+      use(i);
+      return (*i).second.first;
+    }
+
+    return Option<Value>::none();
+  }
+
+private:
+  // Not copyable, not assignable.
+  cache(const cache&);
+  cache& operator = (const cache&);
+
+  // Give the operator access to our internals.
+  friend std::ostream& operator << <>(
+      std::ostream& stream,
+      const cache<Key, Value>& c);
+
+  // Insert key/value into the cache.
+  void insert(const Key& key, const Value& value)
+  {
+    if (keys.size() == capacity) {
+      evict();
+    }
+
+    // Get a "pointer" into the lru list for efficient update.
+    typename list::iterator i = keys.insert(keys.end(), key);
+
+    // Save key/value and "pointer" into lru list.
+    values.insert(std::make_pair(key, std::make_pair(value, i)));
+  }
+
+  // Updates the LRU ordering in the cache for the given iterator.
+  void use(const typename map::iterator& i)
+  {
+    // Move the "pointer" to the end of the lru list.
+    keys.splice(keys.end(), keys, (*i).second.second);
+
+    // Now update the "pointer" so we can do this again.
+    (*i).second.second = --keys.end();
+  }
+
+  // Evict the least-recently used element from the cache.
+  void evict()
+  {
+    const typename map::iterator& i = values.find(keys.front());
+    CHECK(i != values.end());
+    values.erase(i);
+    keys.pop_front();
+  }
+
+  // Size of the cache.
+  int capacity;
+
+  // Cache of values and "pointers" into the least-recently used list.
+  map values;
+
+  // Keys ordered by least-recently used.
+  list keys;
+};
+
+
+template <typename Key, typename Value>
+std::ostream& operator << (
+    std::ostream& stream,
+    const cache<Key, Value>& c)
+{
+  typename cache<Key, Value>::list::const_iterator i1;
+  for (i1 = c.keys.begin(); i1 != c.keys.end(); i1++) {
+    stream << *i1 << ": ";
+    typename cache<Key, Value>::map::const_iterator i2;
+    i2 = c.values.find(*i1);
+    CHECK(i2 != c.values.end());
+    stream << *i2 << std::endl;
+  }
+}
+
+#endif // __CACHE_HPP__

Added: incubator/mesos/trunk/src/common/timer.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/timer.hpp?rev=1230293&view=auto
==============================================================================
--- incubator/mesos/trunk/src/common/timer.hpp (added)
+++ incubator/mesos/trunk/src/common/timer.hpp Wed Jan 11 22:33:29 2012
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.
+ */
+
+#ifndef __TIMER_HPP__
+#define __TIMER_HPP__
+
+#include <time.h>
+
+#ifdef __MACH__
+#include <mach/clock.h>
+#include <mach/mach.h>
+#endif // __MACH__
+
+#include <sys/time.h>
+
+#include "common/seconds.hpp"
+
+class Timer
+{
+public:
+  Timer() : running(false) { started.tv_sec = 0; started.tv_nsec = 0; }
+
+  void start()
+  {
+    started = now();
+    running = true;
+  }
+
+  void stop()
+  {
+    stopped = now();
+    running = false;
+  }
+
+  nanoseconds elapsed()
+  {
+    if (!running) {
+      return nanoseconds(diff(stopped, started));
+    }
+
+    return nanoseconds(diff(now(), started));
+  }
+
+private:
+  static timespec now()
+  {
+    timespec ts;
+#ifdef __MACH__
+    // OS X does not have clock_gettime, use clock_get_time.
+    clock_serv_t cclock;
+    mach_timespec_t mts;
+    host_get_clock_service(mach_host_self(), CALENDAR_CLOCK, &cclock);
+    clock_get_time(cclock, &mts);
+    mach_port_deallocate(mach_task_self(), cclock);
+    ts.tv_sec = mts.tv_sec;
+    ts.tv_nsec = mts.tv_nsec;
+#else
+    clock_gettime(CLOCK_REALTIME, &ts);
+#endif // __MACH__
+    return ts;
+  }
+
+  static uint64_t diff(const timespec& from, const timespec& to)
+  {
+    return ((from.tv_sec - to.tv_sec) * 1000000000LL)
+      + (from.tv_nsec - to.tv_nsec);
+  }
+
+  bool running;
+  timespec started, stopped;
+};
+
+#endif // __TIMER_HPP__

Added: incubator/mesos/trunk/src/common/values.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/values.cpp?rev=1230293&view=auto
==============================================================================
--- incubator/mesos/trunk/src/common/values.cpp (added)
+++ incubator/mesos/trunk/src/common/values.cpp Wed Jan 11 22:33:29 2012
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.
+ */
+
+#include <iostream>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include <boost/lexical_cast.hpp>
+
+#include "common/foreach.hpp"
+#include "common/resources.hpp"
+#include "common/strings.hpp"
+#include "common/values.hpp"
+
+using std::ostream;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace values {
+
+Try<Value> parse(const std::string& text) {
+  Value value;
+
+  // Remove any spaces from the text.
+  string temp;
+  foreach (const char c, text) {
+    if (c != ' ') {
+      temp += c;
+    }
+  }
+
+  if (temp.length() == 0) {
+    return Try<Value>::error(
+      "Error parsing value, expecting non-empty string");
+  }
+
+  // TODO(ynie): Find a better way to check brackets.
+  if (!strings::checkBracketsMatching(temp, '{', '}') ||
+      !strings::checkBracketsMatching(temp, '[', ']') ||
+      !strings::checkBracketsMatching(temp, '(', ')')) {
+    return Try<Value>::error(
+      "Error parsing value, brackets doesn't match");
+  }
+
+  size_t index = temp.find('[');
+  if (index == 0) {
+    // This is a ranges.
+    Value::Ranges ranges;
+    const vector<string>& tokens = strings::split(temp, "[]-,\n");
+    if (tokens.size() % 2 != 0) {
+      return Try<Value>::error("Error parsing value: " + text +
+                               ", expect one or more \"ranges \"");
+    } else {
+      for (int i = 0; i < tokens.size(); i += 2) {
+        Value::Range *range = ranges.add_range();
+
+        int j = i;
+        try {
+          range->set_begin(boost::lexical_cast<uint64_t>((tokens[j++])));
+          range->set_end(boost::lexical_cast<uint64_t>(tokens[j++]));
+        } catch (const boost::bad_lexical_cast&) {
+          return Try<Value>::error(
+            "Error parsing value " + text +
+            ", expecting non-negative integers in '" + tokens[j - 1] + "'");
+        }
+      }
+
+      value.set_type(Value::RANGES);
+      value.mutable_ranges()->MergeFrom(ranges);
+      return Try<Value>::some(value);
+    }
+  } else if (index == string::npos) {
+    size_t index = temp.find('{');
+    if (index == 0) {
+      // This is a set.
+      Value::Set set;
+      const vector<string>& tokens = strings::split(temp, "{},\n");
+      for (int i = 0; i < tokens.size(); i++) {
+        set.add_item(tokens[i]);
+      }
+
+      value.set_type(Value::SET);
+      value.mutable_set()->MergeFrom(set);
+      return Try<Value>::some(value);
+    } else if (index == string::npos) {
+      try {
+        Value::Scalar scalar;
+        scalar.set_value(boost::lexical_cast<double>(temp));
+        // This is a Scalar.
+        value.set_type(Value::SCALAR);
+        value.mutable_scalar()->MergeFrom(scalar);
+        return Try<Value>::some(value);
+      } catch (const boost::bad_lexical_cast&) {
+        // This is a Text.
+        Value::Text text;
+        text.set_value(temp);
+        value.set_type(Value::TEXT);
+        value.mutable_text()->MergeFrom(text);
+        return Try<Value>::some(value);
+      }
+    } else {
+      return Try<Value>::error(
+        "Error parsing value " + text + ", bad '{' found");
+    }
+  } else {
+    return Try<Value>::error(
+      "Error parsing value " + text + ", bad '[' found");
+  }
+
+}
+
+} // namespace values
+} // namespace internal
+} // namespace mesos

Added: incubator/mesos/trunk/src/common/values.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/values.hpp?rev=1230293&view=auto
==============================================================================
--- incubator/mesos/trunk/src/common/values.hpp (added)
+++ incubator/mesos/trunk/src/common/values.hpp Wed Jan 11 22:33:29 2012
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.
+ */
+
+#ifndef __VALUE_PARSER_HPP__
+#define __VALUE_PARSER_HPP__
+
+#include <mesos/mesos.hpp>
+
+#include "common/try.hpp"
+
+namespace mesos {
+namespace internal {
+namespace values {
+
+Try<Value> parse(const std::string& text);
+
+} // namespace values
+} // namespace internal
+} // namespace mesos
+
+#endif // __VALUE_PARSER_HPP__

Added: incubator/mesos/trunk/src/tests/attributes_test.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/attributes_test.cpp?rev=1230293&view=auto
==============================================================================
--- incubator/mesos/trunk/src/tests/attributes_test.cpp (added)
+++ incubator/mesos/trunk/src/tests/attributes_test.cpp Wed Jan 11 22:33:29 2012
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.
+ */
+
+#include <sstream>
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "common/attributes.hpp"
+
+using namespace mesos;
+using namespace mesos::internal;
+
+using std::string;
+
+
+TEST(AttributesTest, Parsing)
+{
+  Attributes a = Attributes::parse("cpus:45.55;"
+                                    "ports:[10000-20000, 30000-50000];"
+                                    "rack:rack1,rack2");
+  ASSERT_EQ(Value::SCALAR, a.get(0).type());
+  ASSERT_EQ("cpus", a.get(0).name());
+  ASSERT_EQ(45.55, a.get(0).scalar().value());
+
+  ASSERT_EQ(Value::RANGES, a.get(1).type());
+  ASSERT_EQ("ports", a.get(1).name());
+  ASSERT_EQ(2, a.get(1).ranges().range_size());
+  ASSERT_EQ(10000, a.get(1).ranges().range(0).begin());
+  ASSERT_EQ(20000, a.get(1).ranges().range(0).end());
+  ASSERT_EQ(30000, a.get(1).ranges().range(1).begin());
+  ASSERT_EQ(50000, a.get(1).ranges().range(1).end());
+
+
+  ASSERT_EQ(Value::TEXT, a.get(2).type());
+  ASSERT_EQ("rack", a.get(2).name());
+  ASSERT_EQ("rack1,rack2", a.get(2).text().value());
+}
\ No newline at end of file

Added: incubator/mesos/trunk/src/tests/values_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/values_tests.cpp?rev=1230293&view=auto
==============================================================================
--- incubator/mesos/trunk/src/tests/values_tests.cpp (added)
+++ incubator/mesos/trunk/src/tests/values_tests.cpp Wed Jan 11 22:33:29 2012
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.
+ */
+
+#include <sstream>
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "common/try.hpp"
+#include "common/values.hpp"
+#include "master/master.hpp"
+
+using namespace mesos;
+using namespace mesos::internal;
+using namespace mesos::internal::values;
+
+using std::string;
+
+
+TEST(ValuesTest, ValidInput)
+{
+  // Test parsing scalar type.
+  Try<Value> result1 = parse("45.55");
+  ASSERT_TRUE(result1.isSome());
+  ASSERT_EQ(Value::SCALAR, result1.get().type());
+  EXPECT_EQ(45.55, result1.get().scalar().value());
+
+  // Test parsing ranges type.
+  Try<Value> result2 = parse("[10000-20000, 30000-50000]");
+  ASSERT_TRUE(result2.isSome());
+  ASSERT_EQ(Value::RANGES, result2.get().type());
+  EXPECT_EQ(2, result2.get().ranges().range_size());
+  EXPECT_EQ(10000, result2.get().ranges().range(0).begin());
+  EXPECT_EQ(20000, result2.get().ranges().range(0).end());
+  EXPECT_EQ(30000, result2.get().ranges().range(1).begin());
+  EXPECT_EQ(50000, result2.get().ranges().range(1).end());
+
+  // Test parsing set type.
+  Try<Value> result3 = parse("{sda1, sda2}");
+  ASSERT_TRUE(result3.isSome());
+  ASSERT_EQ(Value::SET, result3.get().type());
+  ASSERT_EQ(2, result3.get().set().item_size());
+  EXPECT_EQ("sda1", result3.get().set().item(0));
+  EXPECT_EQ("sda2", result3.get().set().item(1));
+
+  // Test parsing text type.
+  Try<Value> result4 = parse("123abc,s");
+  ASSERT_TRUE(result4.isSome());
+  ASSERT_EQ(Value::TEXT, result4.get().type());
+  ASSERT_EQ("123abc,s", result4.get().text().value());
+}
+
+
+TEST(ValuesTest, InvalidInput)
+{
+  // Test when '{' doesn't match.
+  Try<Value> result1 = parse("{aa,b}}");
+  ASSERT_TRUE(result1.isError());
+
+  // Test when '[' doesn't match.
+  Try<Value> result2 = parse("[1-2]]");
+  ASSERT_TRUE(result2.isError());
+
+  // Test when range is not numeric.
+  Try<Value> result3 = parse("[1-2b]");
+  ASSERT_TRUE(result3.isError());
+
+  // Test when giving empty string.
+  Try<Value> result6 = parse("  ");
+  ASSERT_TRUE(result6.isError());
+}
\ No newline at end of file